diff --git a/src/duckdb/extension/parquet/column_reader.cpp b/src/duckdb/extension/parquet/column_reader.cpp index 416857416..959850cf0 100644 --- a/src/duckdb/extension/parquet/column_reader.cpp +++ b/src/duckdb/extension/parquet/column_reader.cpp @@ -20,7 +20,6 @@ #ifndef DUCKDB_AMALGAMATION #include "duckdb/common/types/bit.hpp" #include "duckdb/common/types/blob.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #endif namespace duckdb { @@ -181,11 +180,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::ARRAY) { - return nullptr; - } - return ParquetStatisticsUtils::TransformColumnStatistics(Schema(), Type(), columns[file_idx]); + return ParquetStatisticsUtils::TransformColumnStatistics(*this, columns); } void ColumnReader::Plain(shared_ptr plain_data, uint8_t *defines, idx_t num_values, // NOLINT diff --git a/src/duckdb/extension/parquet/column_writer.cpp b/src/duckdb/extension/parquet/column_writer.cpp index fd942695b..bad4e9fac 100644 --- a/src/duckdb/extension/parquet/column_writer.cpp +++ b/src/duckdb/extension/parquet/column_writer.cpp @@ -13,9 +13,9 @@ #include "duckdb/common/serializer/memory_stream.hpp" #include "duckdb/common/serializer/write_stream.hpp" #include "duckdb/common/string_map_set.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/common/types/date.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/string_heap.hpp" #include "duckdb/common/types/time.hpp" #include "duckdb/common/types/timestamp.hpp" @@ -824,6 +824,22 @@ struct ParquetHugeintOperator { } }; +struct ParquetUhugeintOperator { + template + static TGT Operation(SRC input) { + return Uhugeint::Cast(input); + } + + template + static unique_ptr InitializeStats() { + return make_uniq(); + } + + template + static void HandleStats(ColumnWriterStatistics *stats, SRC source_value, TGT target_value) { + } +}; + template static void TemplatedWritePlain(Vector &col, ColumnWriterStatistics *stats, idx_t chunk_start, idx_t chunk_end, ValidityMask &mask, WriteStream &ser) { @@ -1997,6 +2013,9 @@ unique_ptr ColumnWriter::CreateWriterRecursive(vector>( writer, schema_idx, std::move(schema_path), max_repeat, max_define, can_have_nulls); + case LogicalTypeId::UHUGEINT: + return make_uniq>( + writer, schema_idx, std::move(schema_path), max_repeat, max_define, can_have_nulls); case LogicalTypeId::TIMESTAMP_NS: return make_uniq>( writer, schema_idx, std::move(schema_path), max_repeat, max_define, can_have_nulls); diff --git a/src/duckdb/extension/parquet/include/column_reader.hpp b/src/duckdb/extension/parquet/include/column_reader.hpp index 3d6491ceb..f4e32f391 100644 --- a/src/duckdb/extension/parquet/include/column_reader.hpp +++ b/src/duckdb/extension/parquet/include/column_reader.hpp @@ -19,7 +19,6 @@ #ifndef DUCKDB_AMALGAMATION #include "duckdb/common/operator/cast_operators.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/types/vector_cache.hpp" diff --git a/src/duckdb/extension/parquet/include/decode_utils.hpp b/src/duckdb/extension/parquet/include/decode_utils.hpp index 1fb6bbc22..3b2829d61 100644 --- a/src/duckdb/extension/parquet/include/decode_utils.hpp +++ b/src/duckdb/extension/parquet/include/decode_utils.hpp @@ -7,8 +7,8 @@ class ParquetDecodeUtils { public: template - static T ZigzagToInt(const T n) { - return (n >> 1) ^ -(n & 1); + static T ZigzagToInt(const uint64_t n) { + return T(n >> 1) ^ -T(n & 1); } static const uint64_t BITPACK_MASKS[]; diff --git a/src/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp b/src/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp index 601147a6b..c40c3c026 100644 --- a/src/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp +++ b/src/duckdb/extension/parquet/include/parquet_dbp_decoder.hpp @@ -10,7 +10,7 @@ class DbpDecoder { block_value_count = ParquetDecodeUtils::VarintDecode(buffer_); miniblocks_per_block = ParquetDecodeUtils::VarintDecode(buffer_); total_value_count = ParquetDecodeUtils::VarintDecode(buffer_); - start_value = ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode(buffer_)); + start_value = ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode(buffer_)); // some derivatives D_ASSERT(miniblocks_per_block > 0); @@ -61,7 +61,8 @@ class DbpDecoder { if (bitpack_pos > 0) { // have to eat the leftovers if any buffer_.inc(1); } - min_delta = ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode(buffer_)); + min_delta = + ParquetDecodeUtils::ZigzagToInt(ParquetDecodeUtils::VarintDecode(buffer_)); for (idx_t miniblock_idx = 0; miniblock_idx < miniblocks_per_block; miniblock_idx++) { miniblock_bit_widths[miniblock_idx] = buffer_.read(); // TODO what happens if width is 0? @@ -80,7 +81,7 @@ class DbpDecoder { ParquetDecodeUtils::BitUnpack(buffer_, bitpack_pos, &values[value_offset], read_now, miniblock_bit_widths[miniblock_offset]); for (idx_t i = value_offset; i < value_offset + read_now; i++) { - values[i] = ((i == 0) ? start_value : values[i - 1]) + min_delta + values[i]; + values[i] = T(uint64_t((i == 0) ? start_value : values[i - 1]) + min_delta + uint64_t(values[i])); } value_offset += read_now; values_left_in_miniblock -= read_now; diff --git a/src/duckdb/extension/parquet/include/parquet_decimal_utils.hpp b/src/duckdb/extension/parquet/include/parquet_decimal_utils.hpp index 1789ea970..42debd2c7 100644 --- a/src/duckdb/extension/parquet/include/parquet_decimal_utils.hpp +++ b/src/duckdb/extension/parquet/include/parquet_decimal_utils.hpp @@ -17,18 +17,26 @@ class ParquetDecimalUtils { public: template static PHYSICAL_TYPE ReadDecimalValue(const_data_ptr_t pointer, idx_t size, - const duckdb_parquet::format::SchemaElement &schema_ele) { - D_ASSERT(size <= sizeof(PHYSICAL_TYPE)); + const duckdb_parquet::format::SchemaElement &) { PHYSICAL_TYPE res = 0; auto res_ptr = (uint8_t *)&res; bool positive = (*pointer & 0x80) == 0; // numbers are stored as two's complement so some muckery is required - for (idx_t i = 0; i < size; i++) { + for (idx_t i = 0; i < MinValue(size, sizeof(PHYSICAL_TYPE)); i++) { auto byte = *(pointer + (size - i - 1)); res_ptr[i] = positive ? byte : byte ^ 0xFF; } + // Verify that there are only 0s here + if (size > sizeof(PHYSICAL_TYPE)) { + for (idx_t i = sizeof(PHYSICAL_TYPE); i < size; i++) { + auto byte = *(pointer + (size - i - 1)); + if (byte != 0) { + throw InvalidInputException("Invalid decimal encoding in Parquet file"); + } + } + } if (!positive) { res += 1; return -res; diff --git a/src/duckdb/extension/parquet/include/parquet_statistics.hpp b/src/duckdb/extension/parquet/include/parquet_statistics.hpp index 23d5cf0dc..94ce194e2 100644 --- a/src/duckdb/extension/parquet/include/parquet_statistics.hpp +++ b/src/duckdb/extension/parquet/include/parquet_statistics.hpp @@ -12,11 +12,12 @@ using duckdb_parquet::format::ColumnChunk; using duckdb_parquet::format::SchemaElement; struct LogicalType; +class ColumnReader; struct ParquetStatisticsUtils { - static unique_ptr TransformColumnStatistics(const SchemaElement &s_ele, const LogicalType &type, - const ColumnChunk &column_chunk); + static unique_ptr TransformColumnStatistics(const ColumnReader &reader, + const vector &columns); static Value ConvertValue(const LogicalType &type, const duckdb_parquet::format::SchemaElement &schema_ele, const std::string &stats); diff --git a/src/duckdb/extension/parquet/include/parquet_writer.hpp b/src/duckdb/extension/parquet/include/parquet_writer.hpp index fa6c28c63..6b71b8196 100644 --- a/src/duckdb/extension/parquet/include/parquet_writer.hpp +++ b/src/duckdb/extension/parquet/include/parquet_writer.hpp @@ -87,6 +87,10 @@ class ParquetWriter { BufferedFileWriter &GetWriter() { return *writer; } + idx_t FileSize() { + lock_guard glock(lock); + return writer->total_written; + } static CopyTypeSupport TypeIsSupported(const LogicalType &type); diff --git a/src/duckdb/extension/parquet/parquet_extension.cpp b/src/duckdb/extension/parquet/parquet_extension.cpp index 460c37f80..57906656e 100644 --- a/src/duckdb/extension/parquet/parquet_extension.cpp +++ b/src/duckdb/extension/parquet/parquet_extension.cpp @@ -25,7 +25,6 @@ #include "duckdb/common/multi_file_reader.hpp" #include "duckdb/common/serializer/deserializer.hpp" #include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/function/copy_function.hpp" #include "duckdb/function/pragma_function.hpp" #include "duckdb/function/table_function.hpp" @@ -491,10 +490,9 @@ class ParquetScanFunction { if (bind_data.initial_file_cardinality == 0) { return (100.0 * (bind_data.cur_file + 1)) / bind_data.files.size(); } - auto percentage = (bind_data.chunk_count * STANDARD_VECTOR_SIZE * 100.0 / bind_data.initial_file_cardinality) / - bind_data.files.size(); - percentage += 100.0 * bind_data.cur_file / bind_data.files.size(); - return percentage; + auto percentage = std::min( + 100.0, (bind_data.chunk_count * STANDARD_VECTOR_SIZE * 100.0 / bind_data.initial_file_cardinality)); + return (percentage + 100.0 * bind_data.cur_file) / bind_data.files.size(); } static unique_ptr @@ -630,7 +628,7 @@ class ParquetScanFunction { static idx_t ParquetScanMaxThreads(ClientContext &context, const FunctionData *bind_data) { auto &data = bind_data->Cast(); - return data.initial_file_row_groups * data.files.size(); + return std::max(data.initial_file_row_groups, idx_t(1)) * data.files.size(); } // This function looks for the next available row group. If not available, it will open files from bind_data.files @@ -910,12 +908,12 @@ static void GetFieldIDs(const Value &field_ids_value, ChildFieldIDs &field_ids, } } -unique_ptr ParquetWriteBind(ClientContext &context, const CopyInfo &info, const vector &names, - const vector &sql_types) { +unique_ptr ParquetWriteBind(ClientContext &context, CopyFunctionBindInput &input, + const vector &names, const vector &sql_types) { D_ASSERT(names.size() == sql_types.size()); bool row_group_size_bytes_set = false; auto bind_data = make_uniq(); - for (auto &option : info.options) { + for (auto &option : input.info.options) { const auto loption = StringUtil::Lower(option.first); if (option.second.size() != 1) { // All parquet write options require exactly one argument @@ -986,7 +984,13 @@ unique_ptr ParquetWriteBind(ClientContext &context, const CopyInfo throw NotImplementedException("Unrecognized option for PARQUET: %s", option.first.c_str()); } } - if (!row_group_size_bytes_set) { + if (row_group_size_bytes_set) { + if (DBConfig::GetConfig(context).options.preserve_insertion_order) { + throw BinderException("ROW_GROUP_SIZE_BYTES does not work while preserving insertion order. Use \"SET " + "preserve_insertion_order=false;\" to disable preserving insertion order."); + } + } else { + // We always set a max row group size bytes so we don't use too much memory bind_data->row_group_size_bytes = bind_data->row_group_size * ParquetWriteBindData::BYTES_PER_ROW; } @@ -1179,6 +1183,14 @@ idx_t ParquetWriteDesiredBatchSize(ClientContext &context, FunctionData &bind_da return bind_data.row_group_size; } +//===--------------------------------------------------------------------===// +// Current File Size +//===--------------------------------------------------------------------===// +idx_t ParquetWriteFileSize(GlobalFunctionData &gstate) { + auto &global_state = gstate.Cast(); + return global_state.writer->FileSize(); +} + //===--------------------------------------------------------------------===// // Scan Replacement //===--------------------------------------------------------------------===// @@ -1240,6 +1252,7 @@ void ParquetExtension::Load(DuckDB &db) { function.prepare_batch = ParquetWritePrepareBatch; function.flush_batch = ParquetWriteFlushBatch; function.desired_batch_size = ParquetWriteDesiredBatchSize; + function.file_size_bytes = ParquetWriteFileSize; function.serialize = ParquetCopySerialize; function.deserialize = ParquetCopyDeserialize; function.supports_type = ParquetWriter::TypeIsSupported; diff --git a/src/duckdb/extension/parquet/parquet_reader.cpp b/src/duckdb/extension/parquet/parquet_reader.cpp index ce6b09820..efc16ff0e 100644 --- a/src/duckdb/extension/parquet/parquet_reader.cpp +++ b/src/duckdb/extension/parquet/parquet_reader.cpp @@ -25,6 +25,7 @@ #include "duckdb/planner/filter/conjunction_filter.hpp" #include "duckdb/planner/filter/constant_filter.hpp" #include "duckdb/planner/filter/null_filter.hpp" +#include "duckdb/planner/filter/struct_filter.hpp" #include "duckdb/planner/table_filter.hpp" #include "duckdb/storage/object_cache.hpp" #endif @@ -874,6 +875,11 @@ static void ApplyFilter(Vector &v, TableFilter &filter, parquet_filter_t &filter case TableFilterType::IS_NULL: FilterIsNull(v, filter_mask, count); break; + case TableFilterType::STRUCT_EXTRACT: { + auto &struct_filter = filter.Cast(); + auto &child = StructVector::GetEntries(v)[struct_filter.child_idx]; + ApplyFilter(*child, *struct_filter.child_filter, filter_mask, count); + } break; default: D_ASSERT(0); break; diff --git a/src/duckdb/extension/parquet/parquet_statistics.cpp b/src/duckdb/extension/parquet/parquet_statistics.cpp index a77ace646..351b5938a 100644 --- a/src/duckdb/extension/parquet/parquet_statistics.cpp +++ b/src/duckdb/extension/parquet/parquet_statistics.cpp @@ -4,10 +4,12 @@ #include "parquet_decimal_utils.hpp" #include "parquet_timestamp.hpp" #include "string_column_reader.hpp" +#include "struct_column_reader.hpp" #ifndef DUCKDB_AMALGAMATION #include "duckdb/common/types/blob.hpp" #include "duckdb/common/types/time.hpp" #include "duckdb/common/types/value.hpp" +#include "duckdb/storage/statistics/struct_stats.hpp" #endif namespace duckdb { @@ -127,9 +129,6 @@ Value ParquetStatisticsUtils::ConvertValue(const LogicalType &type, } case Type::BYTE_ARRAY: case Type::FIXED_LEN_BYTE_ARRAY: - if (stats.size() > GetTypeIdSize(type.InternalType())) { - throw InternalException("Incorrect stats size for type %s", type.ToString()); - } switch (type.InternalType()) { case PhysicalType::INT16: return Value::DECIMAL( @@ -209,11 +208,12 @@ Value ParquetStatisticsUtils::ConvertValue(const LogicalType &type, } case LogicalTypeId::TIMESTAMP: case LogicalTypeId::TIMESTAMP_TZ: { + timestamp_t timestamp_value; if (schema_ele.type == Type::INT96) { if (stats.size() != sizeof(Int96)) { throw InternalException("Incorrect stats size for type TIMESTAMP"); } - return Value::TIMESTAMP(ImpalaTimestampToTimestamp(Load(stats_data))); + timestamp_value = ImpalaTimestampToTimestamp(Load(stats_data)); } else { D_ASSERT(schema_ele.type == Type::INT64); if (stats.size() != sizeof(int64_t)) { @@ -223,36 +223,72 @@ Value ParquetStatisticsUtils::ConvertValue(const LogicalType &type, if (schema_ele.__isset.logicalType && schema_ele.logicalType.__isset.TIMESTAMP) { // logical type if (schema_ele.logicalType.TIMESTAMP.unit.__isset.MILLIS) { - return Value::TIMESTAMPMS(timestamp_t(val)); + timestamp_value = Timestamp::FromEpochMs(val); } else if (schema_ele.logicalType.TIMESTAMP.unit.__isset.NANOS) { - return Value::TIMESTAMPNS(timestamp_t(val)); + timestamp_value = Timestamp::FromEpochNanoSeconds(val); } else if (schema_ele.logicalType.TIMESTAMP.unit.__isset.MICROS) { - return Value::TIMESTAMP(timestamp_t(val)); + timestamp_value = timestamp_t(val); } else { throw InternalException("Timestamp logicalType is set but unit is not defined"); } - } - if (schema_ele.converted_type == duckdb_parquet::format::ConvertedType::TIMESTAMP_MILLIS) { - return Value::TIMESTAMPMS(timestamp_t(val)); + } else if (schema_ele.converted_type == duckdb_parquet::format::ConvertedType::TIMESTAMP_MILLIS) { + timestamp_value = Timestamp::FromEpochMs(val); } else { - return Value::TIMESTAMP(timestamp_t(val)); + timestamp_value = timestamp_t(val); } } + if (type.id() == LogicalTypeId::TIMESTAMP_TZ) { + return Value::TIMESTAMPTZ(timestamp_value); + } else { + return Value::TIMESTAMP(timestamp_value); + } } default: throw InternalException("Unsupported type for stats %s", type.ToString()); } } -unique_ptr ParquetStatisticsUtils::TransformColumnStatistics(const SchemaElement &s_ele, - const LogicalType &type, - const ColumnChunk &column_chunk) { +unique_ptr ParquetStatisticsUtils::TransformColumnStatistics(const ColumnReader &reader, + const vector &columns) { + + // Not supported types + if (reader.Type().id() == LogicalTypeId::ARRAY || reader.Type().id() == LogicalTypeId::MAP || + reader.Type().id() == LogicalTypeId::LIST) { + return nullptr; + } + + unique_ptr row_group_stats; + + // Structs are handled differently (they dont have stats) + if (reader.Type().id() == LogicalTypeId::STRUCT) { + auto struct_stats = StructStats::CreateUnknown(reader.Type()); + auto &struct_reader = reader.Cast(); + // Recurse into child readers + for (idx_t i = 0; i < struct_reader.child_readers.size(); i++) { + auto &child_reader = *struct_reader.child_readers[i]; + auto child_stats = ParquetStatisticsUtils::TransformColumnStatistics(child_reader, columns); + StructStats::SetChildStats(struct_stats, i, std::move(child_stats)); + } + row_group_stats = struct_stats.ToUnique(); + + // null count is generic + if (row_group_stats) { + row_group_stats->Set(StatsInfo::CAN_HAVE_NULL_AND_VALID_VALUES); + } + return row_group_stats; + } + + // Otherwise, its a standard column with stats + + auto &column_chunk = columns[reader.FileIdx()]; if (!column_chunk.__isset.meta_data || !column_chunk.meta_data.__isset.statistics) { // no stats present for row group return nullptr; } auto &parquet_stats = column_chunk.meta_data.statistics; - unique_ptr row_group_stats; + + auto &type = reader.Type(); + auto &s_ele = reader.Schema(); switch (type.id()) { case LogicalTypeId::UTINYINT: @@ -267,7 +303,9 @@ unique_ptr ParquetStatisticsUtils::TransformColumnStatistics(con case LogicalTypeId::DOUBLE: case LogicalTypeId::DATE: case LogicalTypeId::TIME: + case LogicalTypeId::TIME_TZ: case LogicalTypeId::TIMESTAMP: + case LogicalTypeId::TIMESTAMP_TZ: case LogicalTypeId::TIMESTAMP_SEC: case LogicalTypeId::TIMESTAMP_MS: case LogicalTypeId::TIMESTAMP_NS: @@ -305,13 +343,11 @@ unique_ptr ParquetStatisticsUtils::TransformColumnStatistics(con } // end of type switch // null count is generic - if (!row_group_stats) { - // if stats are missing from any row group we know squat - return nullptr; - } - row_group_stats->Set(StatsInfo::CAN_HAVE_NULL_AND_VALID_VALUES); - if (parquet_stats.__isset.null_count && parquet_stats.null_count == 0) { - row_group_stats->Set(StatsInfo::CANNOT_HAVE_NULL_VALUES); + if (row_group_stats) { + row_group_stats->Set(StatsInfo::CAN_HAVE_NULL_AND_VALID_VALUES); + if (parquet_stats.__isset.null_count && parquet_stats.null_count == 0) { + row_group_stats->Set(StatsInfo::CANNOT_HAVE_NULL_VALUES); + } } return row_group_stats; } diff --git a/src/duckdb/extension/parquet/parquet_writer.cpp b/src/duckdb/extension/parquet/parquet_writer.cpp index 4ceab3397..0999a8aa8 100644 --- a/src/duckdb/extension/parquet/parquet_writer.cpp +++ b/src/duckdb/extension/parquet/parquet_writer.cpp @@ -101,6 +101,7 @@ CopyTypeSupport ParquetWriter::DuckDBTypeToParquetTypeInternal(const LogicalType case LogicalTypeId::DOUBLE: parquet_type = Type::DOUBLE; break; + case LogicalTypeId::UHUGEINT: case LogicalTypeId::HUGEINT: parquet_type = Type::DOUBLE; return CopyTypeSupport::LOSSY; diff --git a/src/duckdb/src/catalog/catalog.cpp b/src/duckdb/src/catalog/catalog.cpp index 7a4a8c25b..55ee7a25f 100644 --- a/src/duckdb/src/catalog/catalog.cpp +++ b/src/duckdb/src/catalog/catalog.cpp @@ -18,6 +18,7 @@ #include "duckdb/parser/parsed_data/create_copy_function_info.hpp" #include "duckdb/parser/parsed_data/create_index_info.hpp" #include "duckdb/parser/parsed_data/create_pragma_function_info.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" #include "duckdb/parser/parsed_data/create_scalar_function_info.hpp" #include "duckdb/parser/parsed_data/create_schema_info.hpp" #include "duckdb/parser/parsed_data/create_sequence_info.hpp" @@ -300,7 +301,7 @@ struct CatalogLookup { struct CatalogEntryLookup { optional_ptr schema; optional_ptr entry; - PreservedError error; + ErrorData error; DUCKDB_API bool Found() const { return entry; @@ -468,9 +469,9 @@ void Catalog::AutoloadExtensionByConfigName(ClientContext &context, const string throw Catalog::UnrecognizedConfigurationError(context, configuration_name); } -bool Catalog::AutoLoadExtensionByCatalogEntry(ClientContext &context, CatalogType type, const string &entry_name) { +bool Catalog::AutoLoadExtensionByCatalogEntry(DatabaseInstance &db, CatalogType type, const string &entry_name) { #ifndef DUCKDB_DISABLE_EXTENSION_LOAD - auto &dbconfig = DBConfig::GetConfig(context); + auto &dbconfig = DBConfig::GetConfig(db); if (dbconfig.options.autoload_known_extensions) { string extension_name; if (type == CatalogType::TABLE_FUNCTION_ENTRY || type == CatalogType::SCALAR_FUNCTION_ENTRY || @@ -485,7 +486,7 @@ bool Catalog::AutoLoadExtensionByCatalogEntry(ClientContext &context, CatalogTyp } if (!extension_name.empty() && ExtensionHelper::CanAutoloadExtension(extension_name)) { - ExtensionHelper::AutoLoadExtension(context, extension_name); + ExtensionHelper::AutoLoadExtension(db, extension_name); return true; } } @@ -509,9 +510,7 @@ CatalogException Catalog::UnrecognizedConfigurationError(ClientContext &context, for (auto &entry : DBConfig::GetConfig(context).extension_parameters) { potential_names.push_back(entry.first); } - - throw CatalogException("unrecognized configuration parameter \"%s\"\n%s", name, - StringUtil::CandidatesErrorMessage(potential_names, name, "Did you mean")); + throw CatalogException::MissingEntry("configuration parameter", name, potential_names); } CatalogException Catalog::CreateMissingEntryException(ClientContext &context, const string &entry_name, @@ -561,26 +560,24 @@ CatalogException Catalog::CreateMissingEntryException(ClientContext &context, co bool qualify_database; bool qualify_schema; FindMinimalQualification(context, catalog_name, schema_name, qualify_database, qualify_schema); - did_you_mean = "\nDid you mean \"" + unseen_entry.GetQualifiedName(qualify_database, qualify_schema) + "\"?"; + did_you_mean = unseen_entry.GetQualifiedName(qualify_database, qualify_schema); } else if (entry.Found()) { - did_you_mean = "\nDid you mean \"" + entry.name + "\"?"; + did_you_mean = entry.name; } - - return CatalogException(error_context.FormatError("%s with name %s does not exist!%s", CatalogTypeToString(type), - entry_name, did_you_mean)); + return CatalogException::MissingEntry(type, entry_name, did_you_mean, error_context); } CatalogEntryLookup Catalog::TryLookupEntryInternal(CatalogTransaction transaction, CatalogType type, const string &schema, const string &name) { auto schema_entry = GetSchema(transaction, schema, OnEntryNotFound::RETURN_NULL); if (!schema_entry) { - return {nullptr, nullptr, PreservedError()}; + return {nullptr, nullptr, ErrorData()}; } auto entry = schema_entry->GetEntry(transaction, type, name); if (!entry) { - return {schema_entry, nullptr, PreservedError()}; + return {schema_entry, nullptr, ErrorData()}; } - return {schema_entry, entry, PreservedError()}; + return {schema_entry, entry, ErrorData()}; } CatalogEntryLookup Catalog::TryLookupEntry(ClientContext &context, CatalogType type, const string &schema, @@ -613,10 +610,10 @@ CatalogEntryLookup Catalog::TryLookupEntry(ClientContext &context, CatalogType t } if (if_not_found == OnEntryNotFound::RETURN_NULL) { - return {nullptr, nullptr, PreservedError()}; + return {nullptr, nullptr, ErrorData()}; } else { auto except = CreateMissingEntryException(context, name, type, schemas, error_context); - return {nullptr, nullptr, PreservedError(except)}; + return {nullptr, nullptr, ErrorData(except)}; } } @@ -625,7 +622,7 @@ CatalogEntryLookup Catalog::LookupEntry(ClientContext &context, CatalogType type QueryErrorContext error_context) { auto res = TryLookupEntry(context, type, schema, name, if_not_found, error_context); - if (res.error) { + if (res.error.HasError()) { res.error.Throw(); } @@ -648,10 +645,10 @@ CatalogEntryLookup Catalog::TryLookupEntry(ClientContext &context, vector Catalog::GetEntry(ClientContext &context, CatalogType // Try autoloading extension to resolve lookup if (!lookup_entry.Found()) { - if (AutoLoadExtensionByCatalogEntry(context, type, name)) { + if (AutoLoadExtensionByCatalogEntry(*context.db, type, name)) { lookup_entry = TryLookupEntry(context, type, schema_name, name, if_not_found, error_context); } } - if (lookup_entry.error) { + if (lookup_entry.error.HasError()) { lookup_entry.error.Throw(); } @@ -719,12 +716,12 @@ optional_ptr Catalog::GetEntry(ClientContext &context, CatalogType // Try autoloading extension to resolve lookup if (!result.Found()) { - if (AutoLoadExtensionByCatalogEntry(context, type, name)) { + if (AutoLoadExtensionByCatalogEntry(*context.db, type, name)) { result = TryLookupEntry(context, type, catalog, schema, name, if_not_found, error_context); } } - if (result.error) { + if (result.error.HasError()) { result.error.Throw(); } @@ -829,6 +826,7 @@ vector> Catalog::GetAllSchemas(ClientContext &cont void Catalog::Alter(ClientContext &context, AlterInfo &info) { ModifyCatalog(); + auto lookup = LookupEntry(context, info.GetCatalogType(), info.schema, info.name, info.if_not_found); if (!lookup.Found()) { diff --git a/src/duckdb/src/catalog/catalog_entry/column_dependency_manager.cpp b/src/duckdb/src/catalog/catalog_entry/column_dependency_manager.cpp index e09f73dfa..b8b2f0e87 100644 --- a/src/duckdb/src/catalog/catalog_entry/column_dependency_manager.cpp +++ b/src/duckdb/src/catalog/catalog_entry/column_dependency_manager.cpp @@ -2,6 +2,7 @@ #include "duckdb/parser/column_definition.hpp" #include "duckdb/common/set.hpp" #include "duckdb/common/queue.hpp" +#include "duckdb/common/exception/binder_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp b/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp index f5f3d3d72..7230179ba 100644 --- a/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/duck_index_entry.cpp @@ -4,28 +4,54 @@ namespace duckdb { -DuckIndexEntry::DuckIndexEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateIndexInfo &info) - : IndexCatalogEntry(catalog, schema, info) { +IndexDataTableInfo::IndexDataTableInfo(shared_ptr &info_p, const string &index_name_p) + : info(info_p), index_name(index_name_p) { } -DuckIndexEntry::~DuckIndexEntry() { +IndexDataTableInfo::~IndexDataTableInfo() { if (!info) { return; } - info->indexes.RemoveIndex(name); + info->indexes.RemoveIndex(index_name); +} + +DuckIndexEntry::DuckIndexEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateIndexInfo &info) + : IndexCatalogEntry(catalog, schema, info) { +} + +unique_ptr DuckIndexEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + + auto result = make_uniq(catalog, schema, cast_info); + result->info = info; + result->initial_index_size = initial_index_size; + + for (auto &expr : expressions) { + result->expressions.push_back(expr->Copy()); + } + for (auto &expr : parsed_expressions) { + result->parsed_expressions.push_back(expr->Copy()); + } + + return std::move(result); } string DuckIndexEntry::GetSchemaName() const { - return info->schema; + return GetDataTableInfo().schema; } string DuckIndexEntry::GetTableName() const { - return info->table; + return GetDataTableInfo().table; +} + +DataTableInfo &DuckIndexEntry::GetDataTableInfo() const { + return *info->info; } void DuckIndexEntry::CommitDrop() { D_ASSERT(info); - info->indexes.CommitDrop(name); + GetDataTableInfo().indexes.CommitDrop(name); } } // namespace duckdb diff --git a/src/duckdb/src/catalog/catalog_entry/duck_schema_entry.cpp b/src/duckdb/src/catalog/catalog_entry/duck_schema_entry.cpp index b5fbe82da..4a2751ea9 100644 --- a/src/duckdb/src/catalog/catalog_entry/duck_schema_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/duck_schema_entry.cpp @@ -63,14 +63,22 @@ void FindForeignKeyInformation(CatalogEntry &entry, AlterForeignKeyType alter_fk } } -DuckSchemaEntry::DuckSchemaEntry(Catalog &catalog, string name_p, bool is_internal) - : SchemaCatalogEntry(catalog, std::move(name_p), is_internal), - tables(catalog, make_uniq(catalog, *this)), indexes(catalog), table_functions(catalog), - copy_functions(catalog), pragma_functions(catalog), +DuckSchemaEntry::DuckSchemaEntry(Catalog &catalog, CreateSchemaInfo &info) + : SchemaCatalogEntry(catalog, info), tables(catalog, make_uniq(catalog, *this)), + indexes(catalog), table_functions(catalog), copy_functions(catalog), pragma_functions(catalog), functions(catalog, make_uniq(catalog, *this)), sequences(catalog), collations(catalog), types(catalog, make_uniq(catalog, *this)) { } +unique_ptr DuckSchemaEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + + auto result = make_uniq(catalog, cast_info); + + return std::move(result); +} + optional_ptr DuckSchemaEntry::AddEntryInternal(CatalogTransaction transaction, unique_ptr entry, OnCreateConflict on_conflict, @@ -97,7 +105,7 @@ optional_ptr DuckSchemaEntry::AddEntryInternal(CatalogTransaction if (!set.CreateEntry(transaction, entry_name, std::move(entry), dependencies)) { // entry already exists! if (on_conflict == OnCreateConflict::ERROR_ON_CONFLICT) { - throw CatalogException("%s with name \"%s\" already exists!", CatalogTypeToString(entry_type), entry_name); + throw CatalogException::EntryAlreadyExists(entry_type, entry_name); } else { return nullptr; } @@ -239,6 +247,7 @@ optional_ptr DuckSchemaEntry::CreatePragmaFunction(CatalogTransact void DuckSchemaEntry::Alter(ClientContext &context, AlterInfo &info) { CatalogType type = info.GetCatalogType(); + auto &set = GetCatalogSet(type); auto transaction = GetCatalogTransaction(context); if (info.type == AlterType::CHANGE_OWNERSHIP) { @@ -248,7 +257,7 @@ void DuckSchemaEntry::Alter(ClientContext &context, AlterInfo &info) { } else { string name = info.name; if (!set.AlterEntry(transaction, name, info)) { - throw CatalogException("Entry with name \"%s\" does not exist!", name); + throw CatalogException::MissingEntry(type, name, string()); } } } diff --git a/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp b/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp index fba64c652..c55d0a10e 100644 --- a/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/duck_table_entry.cpp @@ -21,6 +21,7 @@ #include "duckdb/planner/table_filter.hpp" #include "duckdb/storage/storage_manager.hpp" #include "duckdb/storage/table_storage_info.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -207,6 +208,10 @@ unique_ptr DuckTableEntry::AlterEntry(ClientContext &context, Alte auto &drop_not_null_info = table_info.Cast(); return DropNotNull(context, drop_not_null_info); } + case AlterTableType::SET_COLUMN_COMMENT: { + auto &column_comment_info = table_info.Cast(); + return SetColumnComment(context, column_comment_info); + } default: throw InternalException("Unrecognized alter table type!"); } @@ -244,6 +249,7 @@ unique_ptr DuckTableEntry::RenameColumn(ClientContext &context, Re } auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; for (auto &col : columns.Logical()) { auto copy = col.Copy(); if (rename_idx == col.Logical()) { @@ -316,6 +322,7 @@ unique_ptr DuckTableEntry::AddColumn(ClientContext &context, AddCo auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; for (auto &col : columns.Logical()) { create_info->columns.AddColumn(col.Copy()); @@ -437,6 +444,7 @@ unique_ptr DuckTableEntry::RemoveColumn(ClientContext &context, Re auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; logical_index_set_t removed_columns; if (column_dependency_manager.HasDependents(removed_index)) { @@ -474,6 +482,7 @@ unique_ptr DuckTableEntry::RemoveColumn(ClientContext &context, Re unique_ptr DuckTableEntry::SetDefault(ClientContext &context, SetDefaultInfo &info) { auto create_info = make_uniq(schema, name); + create_info->comment = comment; auto default_idx = GetColumnIndex(info.column_name); if (default_idx.index == COLUMN_IDENTIFIER_ROW_ID) { throw CatalogException("Cannot SET DEFAULT for rowid column"); @@ -505,6 +514,7 @@ unique_ptr DuckTableEntry::SetDefault(ClientContext &context, SetD unique_ptr DuckTableEntry::SetNotNull(ClientContext &context, SetNotNullInfo &info) { auto create_info = make_uniq(schema, name); + create_info->comment = comment; create_info->columns = columns.Copy(); auto not_null_idx = GetColumnIndex(info.column_name); @@ -541,6 +551,7 @@ unique_ptr DuckTableEntry::SetNotNull(ClientContext &context, SetN unique_ptr DuckTableEntry::DropNotNull(ClientContext &context, DropNotNullInfo &info) { auto create_info = make_uniq(schema, name); + create_info->comment = comment; create_info->columns = columns.Copy(); auto not_null_idx = GetColumnIndex(info.column_name); @@ -566,6 +577,7 @@ unique_ptr DuckTableEntry::ChangeColumnType(ClientContext &context auto change_idx = GetColumnIndex(info.column_name); auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; for (auto &col : columns.Logical()) { auto copy = col.Copy(); @@ -649,10 +661,38 @@ unique_ptr DuckTableEntry::ChangeColumnType(ClientContext &context return std::move(result); } +unique_ptr DuckTableEntry::SetColumnComment(ClientContext &context, SetColumnCommentInfo &info) { + auto create_info = make_uniq(schema, name); + create_info->comment = comment; + auto default_idx = GetColumnIndex(info.column_name); + if (default_idx.index == COLUMN_IDENTIFIER_ROW_ID) { + throw CatalogException("Cannot SET DEFAULT for rowid column"); + } + + // Copy all the columns, changing the value of the one that was specified by 'column_name' + for (auto &col : columns.Logical()) { + auto copy = col.Copy(); + if (default_idx == col.Logical()) { + copy.SetComment(info.comment); + } + create_info->columns.AddColumn(std::move(copy)); + } + // Copy all the constraints + for (idx_t i = 0; i < constraints.size(); i++) { + auto constraint = constraints[i]->Copy(); + create_info->constraints.push_back(std::move(constraint)); + } + + auto binder = Binder::CreateBinder(context); + auto bound_create_info = binder->BindCreateTableInfo(std::move(create_info), schema); + return make_uniq(catalog, schema, *bound_create_info, storage); +} + unique_ptr DuckTableEntry::AddForeignKeyConstraint(ClientContext &context, AlterForeignKeyInfo &info) { D_ASSERT(info.type == AlterForeignKeyType::AFT_ADD); auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; create_info->columns = columns.Copy(); for (idx_t i = 0; i < constraints.size(); i++) { @@ -677,6 +717,7 @@ unique_ptr DuckTableEntry::DropForeignKeyConstraint(ClientContext D_ASSERT(info.type == AlterForeignKeyType::AFT_DELETE); auto create_info = make_uniq(schema, name); create_info->temporary = temporary; + create_info->comment = comment; create_info->columns = columns.Copy(); for (idx_t i = 0; i < constraints.size(); i++) { @@ -698,6 +739,7 @@ unique_ptr DuckTableEntry::DropForeignKeyConstraint(ClientContext unique_ptr DuckTableEntry::Copy(ClientContext &context) const { auto create_info = make_uniq(schema, name); + create_info->comment = comment; create_info->columns = columns.Copy(); for (idx_t i = 0; i < constraints.size(); i++) { diff --git a/src/duckdb/src/catalog/catalog_entry/index_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/index_catalog_entry.cpp index e49dde365..b6da579bc 100644 --- a/src/duckdb/src/catalog/catalog_entry/index_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/index_catalog_entry.cpp @@ -7,6 +7,7 @@ IndexCatalogEntry::IndexCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schem index_type(info.index_type), index_constraint_type(info.constraint_type), column_ids(info.column_ids) { this->temporary = info.temporary; + this->comment = info.comment; } unique_ptr IndexCatalogEntry::GetInfo() const { @@ -28,6 +29,8 @@ unique_ptr IndexCatalogEntry::GetInfo() const { result->parsed_expressions.push_back(expr->Copy()); } + result->comment = comment; + return std::move(result); } diff --git a/src/duckdb/src/catalog/catalog_entry/macro_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/macro_catalog_entry.cpp index 63ea28e3f..3ac57460b 100644 --- a/src/duckdb/src/catalog/catalog_entry/macro_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/macro_catalog_entry.cpp @@ -11,22 +11,38 @@ MacroCatalogEntry::MacroCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schem function(std::move(info.function)) { this->temporary = info.temporary; this->internal = info.internal; + this->comment = info.comment; } ScalarMacroCatalogEntry::ScalarMacroCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateMacroInfo &info) : MacroCatalogEntry(catalog, schema, info) { } +unique_ptr ScalarMacroCatalogEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + auto result = make_uniq(catalog, schema, cast_info); + return std::move(result); +} + TableMacroCatalogEntry::TableMacroCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateMacroInfo &info) : MacroCatalogEntry(catalog, schema, info) { } +unique_ptr TableMacroCatalogEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + auto result = make_uniq(catalog, schema, cast_info); + return std::move(result); +} + unique_ptr MacroCatalogEntry::GetInfo() const { auto info = make_uniq(type); info->catalog = catalog.GetName(); info->schema = schema.name; info->name = name; info->function = function->Copy(); + info->comment = comment; return std::move(info); } 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 c2e3a7279..0f7e1dfc8 100644 --- a/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/schema_catalog_entry.cpp @@ -10,9 +10,10 @@ namespace duckdb { -SchemaCatalogEntry::SchemaCatalogEntry(Catalog &catalog, string name_p, bool internal) - : InCatalogEntry(CatalogType::SCHEMA_ENTRY, catalog, std::move(name_p)) { - this->internal = internal; +SchemaCatalogEntry::SchemaCatalogEntry(Catalog &catalog, CreateSchemaInfo &info) + : InCatalogEntry(CatalogType::SCHEMA_ENTRY, catalog, info.schema) { + this->internal = info.internal; + this->comment = info.comment; } CatalogTransaction SchemaCatalogEntry::GetCatalogTransaction(ClientContext &context) { @@ -35,6 +36,7 @@ SimilarCatalogEntry SchemaCatalogEntry::GetSimilarEntry(CatalogTransaction trans unique_ptr SchemaCatalogEntry::GetInfo() const { auto result = make_uniq(); result->schema = name; + result->comment = comment; return std::move(result); } diff --git a/src/duckdb/src/catalog/catalog_entry/sequence_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/sequence_catalog_entry.cpp index 928335bc7..936fae038 100644 --- a/src/duckdb/src/catalog/catalog_entry/sequence_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/sequence_catalog_entry.cpp @@ -1,44 +1,118 @@ #include "duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp" +#include "duckdb/catalog/catalog.hpp" #include "duckdb/catalog/catalog_entry/schema_catalog_entry.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/parser/parsed_data/create_sequence_info.hpp" #include "duckdb/catalog/dependency_manager.hpp" +#include "duckdb/common/operator/add.hpp" +#include "duckdb/transaction/duck_transaction.hpp" #include #include namespace duckdb { +SequenceData::SequenceData(CreateSequenceInfo &info) + : usage_count(info.usage_count), counter(info.start_value), increment(info.increment), + start_value(info.start_value), min_value(info.min_value), max_value(info.max_value), cycle(info.cycle) { +} + SequenceCatalogEntry::SequenceCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateSequenceInfo &info) - : StandardEntry(CatalogType::SEQUENCE_ENTRY, schema, catalog, info.name), usage_count(info.usage_count), - counter(info.start_value), increment(info.increment), start_value(info.start_value), min_value(info.min_value), - max_value(info.max_value), cycle(info.cycle) { + : StandardEntry(CatalogType::SEQUENCE_ENTRY, schema, catalog, info.name), data(info) { this->temporary = info.temporary; + this->comment = info.comment; +} + +unique_ptr SequenceCatalogEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + + auto result = make_uniq(catalog, schema, cast_info); + result->data = GetData(); + + return std::move(result); +} + +SequenceData SequenceCatalogEntry::GetData() const { + lock_guard seqlock(lock); + return data; +} + +int64_t SequenceCatalogEntry::CurrentValue() { + lock_guard seqlock(lock); + int64_t result; + if (data.usage_count == 0u) { + throw SequenceException("currval: sequence is not yet defined in this session"); + } + result = data.last_value; + return result; +} + +int64_t SequenceCatalogEntry::NextValue(DuckTransaction &transaction) { + lock_guard seqlock(lock); + int64_t result; + result = data.counter; + bool overflow = !TryAddOperator::Operation(data.counter, data.increment, data.counter); + if (data.cycle) { + if (overflow) { + data.counter = data.increment < 0 ? data.max_value : data.min_value; + } else if (data.counter < data.min_value) { + data.counter = data.max_value; + } else if (data.counter > data.max_value) { + data.counter = data.min_value; + } + } else { + if (result < data.min_value || (overflow && data.increment < 0)) { + throw SequenceException("nextval: reached minimum value of sequence \"%s\" (%lld)", name, data.min_value); + } + if (result > data.max_value || overflow) { + throw SequenceException("nextval: reached maximum value of sequence \"%s\" (%lld)", name, data.max_value); + } + } + data.last_value = result; + data.usage_count++; + if (!temporary) { + transaction.sequence_usage[this] = SequenceValue(data.usage_count, data.counter); + } + return result; +} + +void SequenceCatalogEntry::ReplayValue(uint64_t v_usage_count, int64_t v_counter) { + if (v_usage_count > data.usage_count) { + data.usage_count = v_usage_count; + data.counter = v_counter; + } } unique_ptr SequenceCatalogEntry::GetInfo() const { + auto seq_data = GetData(); + auto result = make_uniq(); + result->catalog = catalog.GetName(); result->schema = schema.name; result->name = name; - result->usage_count = usage_count; - result->increment = increment; - result->min_value = min_value; - result->max_value = max_value; - result->start_value = counter; - result->cycle = cycle; + result->usage_count = seq_data.usage_count; + result->increment = seq_data.increment; + result->min_value = seq_data.min_value; + result->max_value = seq_data.max_value; + result->start_value = seq_data.counter; + result->cycle = seq_data.cycle; + result->comment = comment; return std::move(result); } string SequenceCatalogEntry::ToSQL() const { + auto seq_data = GetData(); + std::stringstream ss; ss << "CREATE SEQUENCE "; ss << name; - ss << " INCREMENT BY " << increment; - ss << " MINVALUE " << min_value; - ss << " MAXVALUE " << max_value; - ss << " START " << counter; - ss << " " << (cycle ? "CYCLE" : "NO CYCLE") << ";"; + ss << " INCREMENT BY " << seq_data.increment; + ss << " MINVALUE " << seq_data.min_value; + ss << " MAXVALUE " << seq_data.max_value; + ss << " START " << seq_data.counter; + ss << " " << (seq_data.cycle ? "CYCLE" : "NO CYCLE") << ";"; return ss.str(); } } // 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 eabcab062..e328fc9a4 100644 --- a/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/table_catalog_entry.cpp @@ -21,6 +21,7 @@ TableCatalogEntry::TableCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schem : StandardEntry(CatalogType::TABLE_ENTRY, schema, catalog, info.table), columns(std::move(info.columns)), constraints(std::move(info.constraints)) { this->temporary = info.temporary; + this->comment = info.comment; } bool TableCatalogEntry::HasGeneratedColumns() const { @@ -63,6 +64,7 @@ unique_ptr TableCatalogEntry::GetInfo() const { result->constraints.reserve(constraints.size()); std::for_each(constraints.begin(), constraints.end(), [&result](const unique_ptr &c) { result->constraints.emplace_back(c->Copy()); }); + result->comment = comment; return std::move(result); } 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 f16d85ced..53954feda 100644 --- a/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/type_catalog_entry.cpp @@ -13,6 +13,14 @@ TypeCatalogEntry::TypeCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, : StandardEntry(CatalogType::TYPE_ENTRY, schema, catalog, info.name), user_type(info.type) { this->temporary = info.temporary; this->internal = info.internal; + this->comment = info.comment; +} + +unique_ptr TypeCatalogEntry::Copy(ClientContext &context) const { + auto info_copy = GetInfo(); + auto &cast_info = info_copy->Cast(); + auto result = make_uniq(catalog, schema, cast_info); + return std::move(result); } unique_ptr TypeCatalogEntry::GetInfo() const { @@ -21,6 +29,7 @@ unique_ptr TypeCatalogEntry::GetInfo() const { result->schema = schema.name; result->name = name; result->type = user_type; + result->comment = comment; return std::move(result); } diff --git a/src/duckdb/src/catalog/catalog_entry/view_catalog_entry.cpp b/src/duckdb/src/catalog/catalog_entry/view_catalog_entry.cpp index 9c3a12cab..0459eb2c0 100644 --- a/src/duckdb/src/catalog/catalog_entry/view_catalog_entry.cpp +++ b/src/duckdb/src/catalog/catalog_entry/view_catalog_entry.cpp @@ -17,6 +17,7 @@ void ViewCatalogEntry::Initialize(CreateViewInfo &info) { this->temporary = info.temporary; this->sql = info.sql; this->internal = info.internal; + this->comment = info.comment; } ViewCatalogEntry::ViewCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateViewInfo &info) @@ -33,6 +34,7 @@ unique_ptr ViewCatalogEntry::GetInfo() const { result->aliases = aliases; result->types = types; result->temporary = temporary; + result->comment = comment; return std::move(result); } diff --git a/src/duckdb/src/catalog/catalog_set.cpp b/src/duckdb/src/catalog/catalog_set.cpp index 9776be6b2..672ed0b34 100644 --- a/src/duckdb/src/catalog/catalog_set.cpp +++ b/src/duckdb/src/catalog/catalog_set.cpp @@ -12,6 +12,8 @@ #include "duckdb/parser/parsed_data/alter_table_info.hpp" #include "duckdb/transaction/duck_transaction.hpp" #include "duckdb/transaction/transaction_manager.hpp" +#include "duckdb/catalog/dependency_list.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -307,12 +309,20 @@ bool CatalogSet::AlterEntry(CatalogTransaction transaction, const string &name, throw InternalException("Cannot AlterEntry without client context"); } - // Use the existing entry to create the altered entry auto &context = *transaction.context; - auto value = entry->AlterEntry(context, alter_info); - if (!value) { - // alter failed, but did not result in an error - return true; + + unique_ptr value; + if (alter_info.type == AlterType::SET_COMMENT) { + // Copy the existing entry; we are only changing metadata here + value = entry->Copy(context); + value->comment = alter_info.Cast().comment_value; + } else { + // Use the existing entry to create the altered entry + value = entry->AlterEntry(context, alter_info); + if (!value) { + // alter failed, but did not result in an error + return true; + } } // Mark this entry as being created by this transaction diff --git a/src/duckdb/src/catalog/catalog_transaction.cpp b/src/duckdb/src/catalog/catalog_transaction.cpp index a99a80287..fbe100d3a 100644 --- a/src/duckdb/src/catalog/catalog_transaction.cpp +++ b/src/duckdb/src/catalog/catalog_transaction.cpp @@ -31,6 +31,10 @@ ClientContext &CatalogTransaction::GetContext() { return *context; } +CatalogTransaction CatalogTransaction::GetSystemCatalogTransaction(ClientContext &context) { + return CatalogTransaction(Catalog::GetSystemCatalog(context), context); +} + CatalogTransaction CatalogTransaction::GetSystemTransaction(DatabaseInstance &db) { return CatalogTransaction(db, 1, 1); } diff --git a/src/duckdb/src/catalog/default/default_functions.cpp b/src/duckdb/src/catalog/default/default_functions.cpp index e1c3d75e7..503cdbc46 100644 --- a/src/duckdb/src/catalog/default/default_functions.cpp +++ b/src/duckdb/src/catalog/default/default_functions.cpp @@ -27,6 +27,11 @@ static DefaultMacro internal_macros[] = { {"pg_catalog", "pg_typeof", {"expression", nullptr}, "lower(typeof(expression))"}, // get the data type of any value + {"pg_catalog", "current_database", {nullptr}, "current_database()"}, // name of current database (called "catalog" in the SQL standard) + {"pg_catalog", "current_query", {nullptr}, "current_query()"}, // the currently executing query (NULL if not inside a plpgsql function) + {"pg_catalog", "current_schema", {nullptr}, "current_schema()"}, // name of current schema + {"pg_catalog", "current_schemas", {"include_implicit"}, "current_schemas(include_implicit)"}, // names of schemas in search path + // privilege functions // {"has_any_column_privilege", {"user", "table", "privilege", nullptr}, "true"}, //boolean //does user have privilege for any column of table {"pg_catalog", "has_any_column_privilege", {"table", "privilege", nullptr}, "true"}, //boolean //does current user have privilege for any column of table @@ -92,7 +97,7 @@ static DefaultMacro internal_macros[] = { {DEFAULT_SCHEMA, "array_pop_front", {"arr", nullptr}, "arr[2:]"}, {DEFAULT_SCHEMA, "array_push_back", {"arr", "e", nullptr}, "list_concat(arr, list_value(e))"}, {DEFAULT_SCHEMA, "array_push_front", {"arr", "e", nullptr}, "list_concat(list_value(e), arr)"}, - {DEFAULT_SCHEMA, "array_to_string", {"arr", "sep", nullptr}, "list_aggr(arr, 'string_agg', sep)"}, + {DEFAULT_SCHEMA, "array_to_string", {"arr", "sep", nullptr}, "list_aggr(arr::varchar[], 'string_agg', sep)"}, {DEFAULT_SCHEMA, "generate_subscripts", {"arr", "dim", nullptr}, "unnest(generate_series(1, array_length(arr, dim)))"}, {DEFAULT_SCHEMA, "fdiv", {"x", "y", nullptr}, "floor(x/y)"}, {DEFAULT_SCHEMA, "fmod", {"x", "y", nullptr}, "(x-y*floor(x/y))"}, @@ -138,6 +143,7 @@ static DefaultMacro internal_macros[] = { {DEFAULT_SCHEMA, "list_first", {"l", nullptr}, "list_aggr(l, 'first')"}, {DEFAULT_SCHEMA, "list_any_value", {"l", nullptr}, "list_aggr(l, 'any_value')"}, {DEFAULT_SCHEMA, "list_kurtosis", {"l", nullptr}, "list_aggr(l, 'kurtosis')"}, + {DEFAULT_SCHEMA, "list_kurtosis_pop", {"l", nullptr}, "list_aggr(l, 'kurtosis_pop')"}, {DEFAULT_SCHEMA, "list_min", {"l", nullptr}, "list_aggr(l, 'min')"}, {DEFAULT_SCHEMA, "list_max", {"l", nullptr}, "list_aggr(l, 'max')"}, {DEFAULT_SCHEMA, "list_product", {"l", nullptr}, "list_aggr(l, 'product')"}, @@ -156,6 +162,9 @@ static DefaultMacro internal_macros[] = { // date functions {DEFAULT_SCHEMA, "date_add", {"date", "interval", nullptr}, "date + interval"}, + // storage helper functions + {DEFAULT_SCHEMA, "get_block_size", {"db_name"}, "(SELECT block_size FROM pragma_database_size() WHERE database_name = db_name)"}, + {nullptr, nullptr, {nullptr}, nullptr} }; diff --git a/src/duckdb/src/catalog/default/default_schemas.cpp b/src/duckdb/src/catalog/default/default_schemas.cpp index 394e1ba2a..bda6f6abc 100644 --- a/src/duckdb/src/catalog/default/default_schemas.cpp +++ b/src/duckdb/src/catalog/default/default_schemas.cpp @@ -1,5 +1,6 @@ #include "duckdb/catalog/default/default_schemas.hpp" #include "duckdb/catalog/catalog_entry/duck_schema_entry.hpp" +#include "duckdb/parser/parsed_data/create_schema_info.hpp" #include "duckdb/common/string_util.hpp" namespace duckdb { @@ -25,7 +26,10 @@ DefaultSchemaGenerator::DefaultSchemaGenerator(Catalog &catalog) : DefaultGenera unique_ptr DefaultSchemaGenerator::CreateDefaultEntry(ClientContext &context, const string &entry_name) { if (GetDefaultSchema(entry_name)) { - return make_uniq_base(catalog, StringUtil::Lower(entry_name), true); + CreateSchemaInfo info; + info.schema = StringUtil::Lower(entry_name); + info.internal = true; + return make_uniq_base(catalog, info); } return nullptr; } diff --git a/src/duckdb/src/catalog/default/default_views.cpp b/src/duckdb/src/catalog/default/default_views.cpp index 720045025..a106d87b0 100644 --- a/src/duckdb/src/catalog/default/default_views.cpp +++ b/src/duckdb/src/catalog/default/default_views.cpp @@ -33,12 +33,12 @@ static DefaultView internal_views[] = { {"pg_catalog", "pg_constraint", "SELECT table_oid*1000000+constraint_index oid, constraint_text conname, schema_oid connamespace, CASE constraint_type WHEN 'CHECK' then 'c' WHEN 'UNIQUE' then 'u' WHEN 'PRIMARY KEY' THEN 'p' WHEN 'FOREIGN KEY' THEN 'f' ELSE 'x' END contype, false condeferrable, false condeferred, true convalidated, table_oid conrelid, 0 contypid, 0 conindid, 0 conparentid, 0 confrelid, NULL confupdtype, NULL confdeltype, NULL confmatchtype, true conislocal, 0 coninhcount, false connoinherit, constraint_column_indexes conkey, NULL confkey, NULL conpfeqop, NULL conppeqop, NULL conffeqop, NULL conexclop, expression conbin FROM duckdb_constraints()"}, {"pg_catalog", "pg_database", "SELECT database_oid oid, database_name datname FROM duckdb_databases()"}, {"pg_catalog", "pg_depend", "SELECT * FROM duckdb_dependencies()"}, - {"pg_catalog", "pg_description", "SELECT NULL objoid, NULL classoid, NULL objsubid, NULL description WHERE 1=0"}, + {"pg_catalog", "pg_description", "SELECT table_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_tables() WHERE NOT internal UNION ALL SELECT table_oid AS objoid, database_oid AS classoid, column_index AS objsubid, comment AS description FROM duckdb_columns() WHERE NOT internal UNION ALL SELECT view_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_views() WHERE NOT internal UNION ALL SELECT index_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_indexes UNION ALL SELECT sequence_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_sequences() UNION ALL SELECT type_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_types() WHERE NOT internal UNION ALL SELECT function_oid AS objoid, database_oid AS classoid, 0 AS objsubid, comment AS description FROM duckdb_functions() WHERE NOT internal;"}, {"pg_catalog", "pg_enum", "SELECT NULL oid, a.type_oid enumtypid, list_position(b.labels, a.elabel) enumsortorder, a.elabel enumlabel FROM (SELECT UNNEST(labels) elabel, type_oid FROM duckdb_types() WHERE logical_type='ENUM') a JOIN duckdb_types() b ON a.type_oid=b.type_oid;"}, {"pg_catalog", "pg_index", "SELECT index_oid indexrelid, table_oid indrelid, 0 indnatts, 0 indnkeyatts, is_unique indisunique, is_primary indisprimary, false indisexclusion, true indimmediate, false indisclustered, true indisvalid, false indcheckxmin, true indisready, true indislive, false indisreplident, NULL::INT[] indkey, NULL::OID[] indcollation, NULL::OID[] indclass, NULL::INT[] indoption, expressions indexprs, NULL indpred FROM duckdb_indexes()"}, {"pg_catalog", "pg_indexes", "SELECT schema_name schemaname, table_name tablename, index_name indexname, NULL \"tablespace\", sql indexdef FROM duckdb_indexes()"}, {"pg_catalog", "pg_namespace", "SELECT oid, schema_name nspname, 0 nspowner, NULL nspacl FROM duckdb_schemas()"}, - {"pg_catalog", "pg_proc", "SELECT f.function_oid oid, function_name proname, s.oid pronamespace, varargs provariadic, function_type = 'aggregate' proisagg, function_type = 'table' proretset, return_type prorettype, parameter_types proargtypes, parameters proargnames FROM duckdb_functions() f LEFT JOIN duckdb_schemas() s USING (database_name, schema_name)"}, + {"pg_catalog", "pg_proc", "SELECT f.function_oid oid, function_name proname, s.oid pronamespace, NULL proowner, NULL prolang, 0 procost, 0 prorows, varargs provariadic, 0 prosupport, CASE function_type WHEN 'aggregate' THEN 'a' ELSE 'f' END prokind, false prosecdef, false proleakproof, false proisstrict, function_type = 'table' proretset, case (stability) when 'CONSISTENT' then 'i' when 'CONSISTENT_WITHIN_QUERY' then 's' when 'VOLATILE' then 'v' end provolatile, 'u' proparallel, length(parameters) pronargs, 0 pronargdefaults, return_type prorettype, parameter_types proargtypes, NULL proallargtypes, NULL proargmodes, parameters proargnames, NULL proargdefaults, NULL protrftypes, NULL prosrc, NULL probin, macro_definition prosqlbody, NULL proconfig, NULL proacl, function_type = 'aggregate' proisagg, FROM duckdb_functions() f LEFT JOIN duckdb_schemas() s USING (database_name, schema_name)"}, {"pg_catalog", "pg_sequence", "SELECT sequence_oid seqrelid, 0 seqtypid, start_value seqstart, increment_by seqincrement, max_value seqmax, min_value seqmin, 0 seqcache, cycle seqcycle FROM duckdb_sequences()"}, {"pg_catalog", "pg_sequences", "SELECT schema_name schemaname, sequence_name sequencename, 'duckdb' sequenceowner, 0 data_type, start_value, min_value, max_value, increment_by, cycle, 0 cache_size, last_value FROM duckdb_sequences()"}, {"pg_catalog", "pg_settings", "SELECT name, value setting, description short_desc, CASE WHEN input_type = 'VARCHAR' THEN 'string' WHEN input_type = 'BOOLEAN' THEN 'bool' WHEN input_type IN ('BIGINT', 'UBIGINT') THEN 'integer' ELSE input_type END vartype FROM duckdb_settings()"}, @@ -49,6 +49,10 @@ static DefaultView internal_views[] = { {"information_schema", "columns", "SELECT database_name table_catalog, schema_name table_schema, table_name, column_name, column_index ordinal_position, column_default, CASE WHEN is_nullable THEN 'YES' ELSE 'NO' END is_nullable, data_type, character_maximum_length, NULL character_octet_length, numeric_precision, numeric_precision_radix, numeric_scale, NULL datetime_precision, NULL interval_type, NULL interval_precision, NULL character_set_catalog, NULL character_set_schema, NULL character_set_name, NULL collation_catalog, NULL collation_schema, NULL collation_name, NULL domain_catalog, NULL domain_schema, NULL domain_name, NULL udt_catalog, NULL udt_schema, NULL udt_name, NULL scope_catalog, NULL scope_schema, NULL scope_name, NULL maximum_cardinality, NULL dtd_identifier, NULL is_self_referencing, NULL is_identity, NULL identity_generation, NULL identity_start, NULL identity_increment, NULL identity_maximum, NULL identity_minimum, NULL identity_cycle, NULL is_generated, NULL generation_expression, NULL is_updatable FROM duckdb_columns;"}, {"information_schema", "schemata", "SELECT database_name catalog_name, schema_name, 'duckdb' schema_owner, NULL default_character_set_catalog, NULL default_character_set_schema, NULL default_character_set_name, sql sql_path FROM duckdb_schemas()"}, {"information_schema", "tables", "SELECT database_name table_catalog, schema_name table_schema, table_name, CASE WHEN temporary THEN 'LOCAL TEMPORARY' ELSE 'BASE TABLE' END table_type, NULL self_referencing_column_name, NULL reference_generation, NULL user_defined_type_catalog, NULL user_defined_type_schema, NULL user_defined_type_name, 'YES' is_insertable_into, 'NO' is_typed, CASE WHEN temporary THEN 'PRESERVE' ELSE NULL END commit_action FROM duckdb_tables() UNION ALL SELECT database_name table_catalog, schema_name table_schema, view_name table_name, 'VIEW' table_type, NULL self_referencing_column_name, NULL reference_generation, NULL user_defined_type_catalog, NULL user_defined_type_schema, NULL user_defined_type_name, 'NO' is_insertable_into, 'NO' is_typed, NULL commit_action FROM duckdb_views;"}, + {"information_schema", "character_sets", "SELECT NULL character_set_catalog, NULL character_set_schema, 'UTF8' character_set_name, 'UCS' character_repertoire, 'UTF8' form_of_use, current_database() default_collate_catalog, 'pg_catalog' default_collate_schema, 'ucs_basic' default_collate_name;"}, + {"information_schema", "referential_constraints", "SELECT f.database_name constraint_catalog, f.schema_name constraint_schema, concat(f.source, '_', f.target, '_', f.target_column, '_fkey') constraint_name, current_database() unique_constraint_catalog, c.schema_name unique_constraint_schema, concat(c.table_name, '_', f.target_column, '_', CASE WHEN c.constraint_type == 'UNIQUE' THEN 'key' ELSE 'pkey' END) unique_constraint_name, 'NONE' match_option, 'NO ACTION' update_rule, 'NO ACTION' delete_rule FROM duckdb_constraints() c JOIN (SELECT *, name_extract['source'] source, name_extract['target'] target, name_extract['target_column'] target_column FROM (SELECT *, regexp_extract(constraint_text, 'FOREIGN KEY \\(([a-zA-Z_0-9]+)\\) REFERENCES ([a-zA-Z_0-9]+)\\(([a-zA-Z_0-9]+)\\)', ['source', 'target', 'target_column']) name_extract FROM duckdb_constraints() WHERE constraint_type = 'FOREIGN KEY')) f ON name_extract['target'] = c.table_name AND (c.constraint_type = 'UNIQUE' OR c.constraint_type = 'PRIMARY KEY')"}, + {"information_schema", "key_column_usage", "SELECT current_database() constraint_catalog, schema_name constraint_schema, concat(table_name, '_', constraint_column_names[1], CASE constraint_type WHEN 'FOREIGN KEY' THEN '_fkey' WHEN 'PRIMARY KEY' THEN '_pkey' ELSE '_key' END) constraint_name, current_database() table_catalog, schema_name table_schema, table_name, constraint_column_names[1] column_name, 1 ordinal_position, CASE constraint_type WHEN 'FOREIGN KEY' THEN 1 ELSE NULL END position_in_unique_constraint FROM duckdb_constraints() WHERE constraint_type = 'FOREIGN KEY' OR constraint_type = 'PRIMARY KEY' OR constraint_type = 'UNIQUE';"}, + {"information_schema", "table_constraints", "SELECT current_database() constraint_catalog, schema_name constraint_schema, concat(table_name, '_', CASE WHEN length(constraint_column_names) > 1 THEN NULL ELSE constraint_column_names[1] || '_' END, CASE constraint_type WHEN 'FOREIGN KEY' THEN 'fkey' WHEN 'PRIMARY KEY' THEN 'pkey' WHEN 'UNIQUE' THEN 'key' WHEN 'CHECK' THEN 'check' WHEN 'NOT NULL' THEN 'not_null' END) constraint_name, current_database() table_catalog, schema_name table_schema, table_name, CASE constraint_type WHEN 'NOT NULL' THEN 'CHECK' ELSE constraint_type END constraint_type, 'NO' is_deferrable, 'NO' initially_deferred, 'YES' enforced, 'YES' nulls_distinct FROM duckdb_constraints() WHERE constraint_type = 'PRIMARY KEY' OR constraint_type = 'FOREIGN KEY' OR constraint_type = 'UNIQUE' OR constraint_type = 'CHECK' OR constraint_type = 'NOT NULL';"}, {nullptr, nullptr, nullptr}}; static unique_ptr GetDefaultView(ClientContext &context, const string &input_schema, const string &input_name) { diff --git a/src/duckdb/src/catalog/duck_catalog.cpp b/src/duckdb/src/catalog/duck_catalog.cpp index 879761d20..8f712b843 100644 --- a/src/duckdb/src/catalog/duck_catalog.cpp +++ b/src/duckdb/src/catalog/duck_catalog.cpp @@ -55,7 +55,7 @@ bool DuckCatalog::IsDuckCatalog() { //===--------------------------------------------------------------------===// optional_ptr DuckCatalog::CreateSchemaInternal(CatalogTransaction transaction, CreateSchemaInfo &info) { DependencyList dependencies; - auto entry = make_uniq(*this, info.schema, info.internal); + auto entry = make_uniq(*this, info); auto result = entry.get(); if (!schemas->CreateEntry(transaction, info.schema, std::move(entry), dependencies)) { return nullptr; @@ -69,7 +69,7 @@ optional_ptr DuckCatalog::CreateSchema(CatalogTransaction transact if (!result) { switch (info.on_conflict) { case OnCreateConflict::ERROR_ON_CONFLICT: - throw CatalogException("Schema with name %s already exists!", info.schema); + throw CatalogException::EntryAlreadyExists(CatalogType::SCHEMA_ENTRY, info.schema); case OnCreateConflict::REPLACE_ON_CONFLICT: { DropInfo drop_info; drop_info.type = CatalogType::SCHEMA_ENTRY; @@ -97,7 +97,7 @@ void DuckCatalog::DropSchema(CatalogTransaction transaction, DropInfo &info) { ModifyCatalog(); if (!schemas->DropEntry(transaction, info.name, info.cascade)) { if (info.if_not_found == OnEntryNotFound::THROW_EXCEPTION) { - throw CatalogException("Schema with name \"%s\" does not exist!", info.name); + throw CatalogException::MissingEntry(CatalogType::SCHEMA_ENTRY, info.name, string()); } } } @@ -121,7 +121,7 @@ optional_ptr DuckCatalog::GetSchema(CatalogTransaction trans auto entry = schemas->GetEntry(transaction, schema_name); if (!entry) { if (if_not_found == OnEntryNotFound::THROW_EXCEPTION) { - throw CatalogException(error_context.FormatError("Schema with name %s does not exist!", schema_name)); + throw CatalogException(error_context, "Schema with name %s does not exist!", schema_name); } return nullptr; } diff --git a/src/duckdb/src/common/adbc/adbc.cpp b/src/duckdb/src/common/adbc/adbc.cpp index 9a771caf4..18d384c87 100644 --- a/src/duckdb/src/common/adbc/adbc.cpp +++ b/src/duckdb/src/common/adbc/adbc.cpp @@ -604,7 +604,8 @@ AdbcStatusCode Ingest(duckdb_connection connection, const char *table_name, stru input->release = nullptr; } catch (std::exception &ex) { if (error) { - error->message = strdup(ex.what()); + ::duckdb::ErrorData parsed_error(ex); + error->message = strdup(parsed_error.RawMessage().c_str()); } return ADBC_STATUS_INTERNAL; } catch (...) { @@ -711,7 +712,8 @@ AdbcStatusCode GetPreparedParameters(duckdb_connection connection, duckdb::uniqu input->release = nullptr; } catch (std::exception &ex) { if (error) { - error->message = strdup(ex.what()); + ::duckdb::ErrorData parsed_error(ex); + error->message = strdup(parsed_error.RawMessage().c_str()); } return ADBC_STATUS_INTERNAL; } catch (...) { diff --git a/src/duckdb/src/common/arrow/arrow_wrapper.cpp b/src/duckdb/src/common/arrow/arrow_wrapper.cpp index 5a39f81b6..be914f74d 100644 --- a/src/duckdb/src/common/arrow/arrow_wrapper.cpp +++ b/src/duckdb/src/common/arrow/arrow_wrapper.cpp @@ -81,7 +81,7 @@ int ResultArrowArrayStreamWrapper::MyStreamGetSchema(struct ArrowArrayStream *st if (result.type == QueryResultType::STREAM_RESULT) { auto &stream_result = result.Cast(); if (!stream_result.IsOpen()) { - my_stream->last_error = PreservedError("Query Stream is closed"); + my_stream->last_error = ErrorData("Query Stream is closed"); return -1; } } @@ -118,10 +118,10 @@ int ResultArrowArrayStreamWrapper::MyStreamGetNext(struct ArrowArrayStream *stre my_stream->column_names = result.names; } idx_t result_count; - PreservedError error; + ErrorData error; if (!ArrowUtil::TryFetchChunk(scan_state, result.client_properties, my_stream->batch_size, out, result_count, error)) { - D_ASSERT(error); + D_ASSERT(error.HasError()); my_stream->last_error = error; return -1; } @@ -166,7 +166,7 @@ ResultArrowArrayStreamWrapper::ResultArrowArrayStreamWrapper(unique_ptr namespace duckdb { diff --git a/src/duckdb/src/common/box_renderer.cpp b/src/duckdb/src/common/box_renderer.cpp index bdb4afa7a..0a64af13e 100644 --- a/src/duckdb/src/common/box_renderer.cpp +++ b/src/duckdb/src/common/box_renderer.cpp @@ -96,6 +96,8 @@ string BoxRenderer::RenderType(const LogicalType &type) { return "uint32"; case LogicalTypeId::UBIGINT: return "uint64"; + case LogicalTypeId::UHUGEINT: + return "uint128"; case LogicalTypeId::LIST: { auto child = RenderType(ListType::GetChildType(type)); return child + "[]"; @@ -116,6 +118,7 @@ ValueRenderAlignment BoxRenderer::TypeAlignment(const LogicalType &type) { case LogicalTypeId::USMALLINT: case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::DECIMAL: case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: @@ -254,7 +257,55 @@ list BoxRenderer::PivotCollections(ClientContext &context, } string ConvertRenderValue(const string &input) { - return StringUtil::Replace(StringUtil::Replace(input, "\n", "\\n"), string("\0", 1), "\\0"); + string result; + result.reserve(input.size()); + for (idx_t c = 0; c < input.size(); c++) { + data_t byte_value = const_data_ptr_cast(input.c_str())[c]; + if (byte_value < 32) { + // ASCII control character + result += "\\"; + switch (input[c]) { + case 7: + // bell + result += 'a'; + break; + case 8: + // backspace + result += 'b'; + break; + case 9: + // tab + result += 't'; + break; + case 10: + // newline + result += 'n'; + break; + case 11: + // vertical tab + result += 'v'; + break; + case 12: + // form feed + result += 'f'; + break; + case 13: + // cariage return + result += 'r'; + break; + case 27: + // escape + result += 'e'; + break; + default: + result += to_string(byte_value); + break; + } + } else { + result += input[c]; + } + } + return result; } string BoxRenderer::GetRenderValue(ColumnDataRowCollection &rows, idx_t c, idx_t r) { diff --git a/src/duckdb/src/common/compressed_file_system.cpp b/src/duckdb/src/common/compressed_file_system.cpp index a9d289330..dab274b27 100644 --- a/src/duckdb/src/common/compressed_file_system.cpp +++ b/src/duckdb/src/common/compressed_file_system.cpp @@ -7,6 +7,7 @@ StreamWrapper::~StreamWrapper() { CompressedFile::CompressedFile(CompressedFileSystem &fs, unique_ptr child_handle_p, const string &path) : FileHandle(fs, path), compressed_fs(fs), child_handle(std::move(child_handle_p)) { + D_ASSERT(child_handle->SeekPosition() == 0); } CompressedFile::~CompressedFile() { diff --git a/src/duckdb/src/common/constants.cpp b/src/duckdb/src/common/constants.cpp index 0e2461c2f..a6c12be12 100644 --- a/src/duckdb/src/common/constants.cpp +++ b/src/duckdb/src/common/constants.cpp @@ -9,7 +9,6 @@ constexpr const idx_t DConstants::INVALID_INDEX; const row_t MAX_ROW_ID = 36028797018960000ULL; // 2^55 const row_t MAX_ROW_ID_LOCAL = 72057594037920000ULL; // 2^56 const column_t COLUMN_IDENTIFIER_ROW_ID = (column_t)-1; -const sel_t ZERO_VECTOR[STANDARD_VECTOR_SIZE] = {0}; const double PI = 3.141592653589793; const transaction_t TRANSACTION_ID_START = 4611686018427388000ULL; // 2^62 diff --git a/src/duckdb/src/common/enum_util.cpp b/src/duckdb/src/common/enum_util.cpp index ab700ac6a..2c5c2568f 100644 --- a/src/duckdb/src/common/enum_util.cpp +++ b/src/duckdb/src/common/enum_util.cpp @@ -70,10 +70,9 @@ #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" -#include "duckdb/execution/operator/scan/csv/csv_option.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state.hpp" -#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_option.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state.hpp" +#include "duckdb/execution/operator/csv_scanner/quote_rules.hpp" #include "duckdb/function/aggregate_state.hpp" #include "duckdb/function/function.hpp" #include "duckdb/function/macro_function.hpp" @@ -86,6 +85,7 @@ #include "duckdb/main/error_manager.hpp" #include "duckdb/main/extension_helper.hpp" #include "duckdb/main/query_result.hpp" +#include "duckdb/main/secret/secret.hpp" #include "duckdb/parallel/interrupt.hpp" #include "duckdb/parallel/task.hpp" #include "duckdb/parser/constraint.hpp" @@ -96,6 +96,7 @@ #include "duckdb/parser/parsed_data/alter_table_function_info.hpp" #include "duckdb/parser/parsed_data/alter_table_info.hpp" #include "duckdb/parser/parsed_data/create_sequence_info.hpp" +#include "duckdb/parser/parsed_data/extra_drop_info.hpp" #include "duckdb/parser/parsed_data/load_info.hpp" #include "duckdb/parser/parsed_data/parse_info.hpp" #include "duckdb/parser/parsed_data/pragma_info.hpp" @@ -107,6 +108,7 @@ #include "duckdb/parser/simplified_token.hpp" #include "duckdb/parser/statement/explain_statement.hpp" #include "duckdb/parser/statement/insert_statement.hpp" +#include "duckdb/parser/tableref/showref.hpp" #include "duckdb/planner/binder.hpp" #include "duckdb/planner/bound_result_modifier.hpp" #include "duckdb/planner/table_filter.hpp" @@ -153,6 +155,29 @@ AccessMode EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(AggregateCombineType value) { + switch(value) { + case AggregateCombineType::PRESERVE_INPUT: + return "PRESERVE_INPUT"; + case AggregateCombineType::ALLOW_DESTRUCTIVE: + return "ALLOW_DESTRUCTIVE"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +AggregateCombineType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "PRESERVE_INPUT")) { + return AggregateCombineType::PRESERVE_INPUT; + } + if (StringUtil::Equals(value, "ALLOW_DESTRUCTIVE")) { + return AggregateCombineType::ALLOW_DESTRUCTIVE; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(AggregateHandling value) { switch(value) { @@ -319,6 +344,8 @@ const char* EnumUtil::ToChars(AlterTableType value) { return "SET_NOT_NULL"; case AlterTableType::DROP_NOT_NULL: return "DROP_NOT_NULL"; + case AlterTableType::SET_COLUMN_COMMENT: + return "SET_COLUMN_COMMENT"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -356,6 +383,9 @@ AlterTableType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "DROP_NOT_NULL")) { return AlterTableType::DROP_NOT_NULL; } + if (StringUtil::Equals(value, "SET_COLUMN_COMMENT")) { + return AlterTableType::SET_COLUMN_COMMENT; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -376,6 +406,8 @@ const char* EnumUtil::ToChars(AlterType value) { return "ALTER_SCALAR_FUNCTION"; case AlterType::ALTER_TABLE_FUNCTION: return "ALTER_TABLE_FUNCTION"; + case AlterType::SET_COMMENT: + return "SET_COMMENT"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -404,6 +436,9 @@ AlterType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "ALTER_TABLE_FUNCTION")) { return AlterType::ALTER_TABLE_FUNCTION; } + if (StringUtil::Equals(value, "SET_COMMENT")) { + return AlterType::SET_COMMENT; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -668,10 +703,14 @@ const char* EnumUtil::ToChars(CSVState value) { return "UNQUOTED"; case CSVState::ESCAPE: return "ESCAPE"; - case CSVState::EMPTY_LINE: - return "EMPTY_LINE"; case CSVState::INVALID: return "INVALID"; + case CSVState::NOT_SET: + return "NOT_SET"; + case CSVState::QUOTED_NEW_LINE: + return "QUOTED_NEW_LINE"; + case CSVState::EMPTY_SPACE: + return "EMPTY_SPACE"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -700,12 +739,18 @@ CSVState EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "ESCAPE")) { return CSVState::ESCAPE; } - if (StringUtil::Equals(value, "EMPTY_LINE")) { - return CSVState::EMPTY_LINE; - } if (StringUtil::Equals(value, "INVALID")) { return CSVState::INVALID; } + if (StringUtil::Equals(value, "NOT_SET")) { + return CSVState::NOT_SET; + } + if (StringUtil::Equals(value, "QUOTED_NEW_LINE")) { + return CSVState::QUOTED_NEW_LINE; + } + if (StringUtil::Equals(value, "EMPTY_SPACE")) { + return CSVState::EMPTY_SPACE; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -778,6 +823,12 @@ const char* EnumUtil::ToChars(CatalogType value) { return "DELETED_ENTRY"; case CatalogType::RENAMED_ENTRY: return "RENAMED_ENTRY"; + case CatalogType::SECRET_ENTRY: + return "SECRET_ENTRY"; + case CatalogType::SECRET_TYPE_ENTRY: + return "SECRET_TYPE_ENTRY"; + case CatalogType::SECRET_FUNCTION_ENTRY: + return "SECRET_FUNCTION_ENTRY"; case CatalogType::DEPENDENCY_ENTRY: return "DEPENDENCY_ENTRY"; default: @@ -844,6 +895,15 @@ CatalogType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "RENAMED_ENTRY")) { return CatalogType::RENAMED_ENTRY; } + if (StringUtil::Equals(value, "SECRET_ENTRY")) { + return CatalogType::SECRET_ENTRY; + } + if (StringUtil::Equals(value, "SECRET_TYPE_ENTRY")) { + return CatalogType::SECRET_TYPE_ENTRY; + } + if (StringUtil::Equals(value, "SECRET_FUNCTION_ENTRY")) { + return CatalogType::SECRET_FUNCTION_ENTRY; + } if (StringUtil::Equals(value, "DEPENDENCY_ENTRY")) { return CatalogType::DEPENDENCY_ENTRY; } @@ -1041,6 +1101,10 @@ const char* EnumUtil::ToChars(CompressionType value) { return "COMPRESSION_CHIMP"; case CompressionType::COMPRESSION_PATAS: return "COMPRESSION_PATAS"; + case CompressionType::COMPRESSION_ALP: + return "COMPRESSION_ALP"; + case CompressionType::COMPRESSION_ALPRD: + return "COMPRESSION_ALPRD"; case CompressionType::COMPRESSION_COUNT: return "COMPRESSION_COUNT"; default: @@ -1080,6 +1144,12 @@ CompressionType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "COMPRESSION_PATAS")) { return CompressionType::COMPRESSION_PATAS; } + if (StringUtil::Equals(value, "COMPRESSION_ALP")) { + return CompressionType::COMPRESSION_ALP; + } + if (StringUtil::Equals(value, "COMPRESSION_ALPRD")) { + return CompressionType::COMPRESSION_ALPRD; + } if (StringUtil::Equals(value, "COMPRESSION_COUNT")) { return CompressionType::COMPRESSION_COUNT; } @@ -1407,6 +1477,34 @@ DependencyEntryType EnumUtil::FromString(const char *value) throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(DeprecatedIndexType value) { + switch(value) { + case DeprecatedIndexType::INVALID: + return "INVALID"; + case DeprecatedIndexType::ART: + return "ART"; + case DeprecatedIndexType::EXTENSION: + return "EXTENSION"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +DeprecatedIndexType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "INVALID")) { + return DeprecatedIndexType::INVALID; + } + if (StringUtil::Equals(value, "ART")) { + return DeprecatedIndexType::ART; + } + if (StringUtil::Equals(value, "EXTENSION")) { + return DeprecatedIndexType::EXTENSION; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(DistinctType value) { switch(value) { @@ -2136,6 +2234,29 @@ ExtensionLoadResult EnumUtil::FromString(const char *value) throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(ExtraDropInfoType value) { + switch(value) { + case ExtraDropInfoType::INVALID: + return "INVALID"; + case ExtraDropInfoType::SECRET_INFO: + return "SECRET_INFO"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +ExtraDropInfoType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "INVALID")) { + return ExtraDropInfoType::INVALID; + } + if (StringUtil::Equals(value, "SECRET_INFO")) { + return ExtraDropInfoType::SECRET_INFO; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(ExtraTypeInfoType value) { switch(value) { @@ -2159,6 +2280,10 @@ const char* EnumUtil::ToChars(ExtraTypeInfoType value) { return "AGGREGATE_STATE_TYPE_INFO"; case ExtraTypeInfoType::ARRAY_TYPE_INFO: return "ARRAY_TYPE_INFO"; + case ExtraTypeInfoType::ANY_TYPE_INFO: + return "ANY_TYPE_INFO"; + case ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO: + return "INTEGER_LITERAL_TYPE_INFO"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -2196,6 +2321,12 @@ ExtraTypeInfoType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "ARRAY_TYPE_INFO")) { return ExtraTypeInfoType::ARRAY_TYPE_INFO; } + if (StringUtil::Equals(value, "ANY_TYPE_INFO")) { + return ExtraTypeInfoType::ANY_TYPE_INFO; + } + if (StringUtil::Equals(value, "INTEGER_LITERAL_TYPE_INFO")) { + return ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -2401,24 +2532,29 @@ FunctionNullHandling EnumUtil::FromString(const char *valu } template<> -const char* EnumUtil::ToChars(FunctionSideEffects value) { +const char* EnumUtil::ToChars(FunctionStability value) { switch(value) { - case FunctionSideEffects::NO_SIDE_EFFECTS: - return "NO_SIDE_EFFECTS"; - case FunctionSideEffects::HAS_SIDE_EFFECTS: - return "HAS_SIDE_EFFECTS"; + case FunctionStability::CONSISTENT: + return "CONSISTENT"; + case FunctionStability::VOLATILE: + return "VOLATILE"; + case FunctionStability::CONSISTENT_WITHIN_QUERY: + return "CONSISTENT_WITHIN_QUERY"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } } template<> -FunctionSideEffects EnumUtil::FromString(const char *value) { - if (StringUtil::Equals(value, "NO_SIDE_EFFECTS")) { - return FunctionSideEffects::NO_SIDE_EFFECTS; +FunctionStability EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "CONSISTENT")) { + return FunctionStability::CONSISTENT; } - if (StringUtil::Equals(value, "HAS_SIDE_EFFECTS")) { - return FunctionSideEffects::HAS_SIDE_EFFECTS; + if (StringUtil::Equals(value, "VOLATILE")) { + return FunctionStability::VOLATILE; + } + if (StringUtil::Equals(value, "CONSISTENT_WITHIN_QUERY")) { + return FunctionStability::CONSISTENT_WITHIN_QUERY; } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -2474,34 +2610,6 @@ IndexConstraintType EnumUtil::FromString(const char *value) throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } -template<> -const char* EnumUtil::ToChars(IndexType value) { - switch(value) { - case IndexType::INVALID: - return "INVALID"; - case IndexType::ART: - return "ART"; - case IndexType::EXTENSION: - return "EXTENSION"; - default: - throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); - } -} - -template<> -IndexType EnumUtil::FromString(const char *value) { - if (StringUtil::Equals(value, "INVALID")) { - return IndexType::INVALID; - } - if (StringUtil::Equals(value, "ART")) { - return IndexType::ART; - } - if (StringUtil::Equals(value, "EXTENSION")) { - return IndexType::EXTENSION; - } - throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); -} - template<> const char* EnumUtil::ToChars(InsertColumnOrder value) { switch(value) { @@ -2832,8 +2940,6 @@ const char* EnumUtil::ToChars(LogicalOperatorType value) { return "LOGICAL_DETACH"; case LogicalOperatorType::LOGICAL_EXPLAIN: return "LOGICAL_EXPLAIN"; - case LogicalOperatorType::LOGICAL_SHOW: - return "LOGICAL_SHOW"; case LogicalOperatorType::LOGICAL_PREPARE: return "LOGICAL_PREPARE"; case LogicalOperatorType::LOGICAL_EXECUTE: @@ -2848,6 +2954,8 @@ const char* EnumUtil::ToChars(LogicalOperatorType value) { return "LOGICAL_LOAD"; case LogicalOperatorType::LOGICAL_RESET: return "LOGICAL_RESET"; + case LogicalOperatorType::LOGICAL_CREATE_SECRET: + return "LOGICAL_CREATE_SECRET"; case LogicalOperatorType::LOGICAL_EXTENSION_OPERATOR: return "LOGICAL_EXTENSION_OPERATOR"; default: @@ -3013,9 +3121,6 @@ LogicalOperatorType EnumUtil::FromString(const char *value) if (StringUtil::Equals(value, "LOGICAL_EXPLAIN")) { return LogicalOperatorType::LOGICAL_EXPLAIN; } - if (StringUtil::Equals(value, "LOGICAL_SHOW")) { - return LogicalOperatorType::LOGICAL_SHOW; - } if (StringUtil::Equals(value, "LOGICAL_PREPARE")) { return LogicalOperatorType::LOGICAL_PREPARE; } @@ -3037,6 +3142,9 @@ LogicalOperatorType EnumUtil::FromString(const char *value) if (StringUtil::Equals(value, "LOGICAL_RESET")) { return LogicalOperatorType::LOGICAL_RESET; } + if (StringUtil::Equals(value, "LOGICAL_CREATE_SECRET")) { + return LogicalOperatorType::LOGICAL_CREATE_SECRET; + } if (StringUtil::Equals(value, "LOGICAL_EXTENSION_OPERATOR")) { return LogicalOperatorType::LOGICAL_EXTENSION_OPERATOR; } @@ -3106,6 +3214,12 @@ const char* EnumUtil::ToChars(LogicalTypeId value) { return "TIME WITH TIME ZONE"; case LogicalTypeId::BIT: return "BIT"; + case LogicalTypeId::STRING_LITERAL: + return "STRING_LITERAL"; + case LogicalTypeId::INTEGER_LITERAL: + return "INTEGER_LITERAL"; + case LogicalTypeId::UHUGEINT: + return "UHUGEINT"; case LogicalTypeId::HUGEINT: return "HUGEINT"; case LogicalTypeId::POINTER: @@ -3229,6 +3343,15 @@ LogicalTypeId EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "BIT")) { return LogicalTypeId::BIT; } + if (StringUtil::Equals(value, "STRING_LITERAL")) { + return LogicalTypeId::STRING_LITERAL; + } + if (StringUtil::Equals(value, "INTEGER_LITERAL")) { + return LogicalTypeId::INTEGER_LITERAL; + } + if (StringUtil::Equals(value, "UHUGEINT")) { + return LogicalTypeId::UHUGEINT; + } if (StringUtil::Equals(value, "HUGEINT")) { return LogicalTypeId::HUGEINT; } @@ -3415,8 +3538,6 @@ const char* EnumUtil::ToChars(NewLineIdentifier value) { return "SINGLE"; case NewLineIdentifier::CARRY_ON: return "CARRY_ON"; - case NewLineIdentifier::MIX: - return "MIX"; case NewLineIdentifier::NOT_SET: return "NOT_SET"; default: @@ -3432,9 +3553,6 @@ NewLineIdentifier EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "CARRY_ON")) { return NewLineIdentifier::CARRY_ON; } - if (StringUtil::Equals(value, "MIX")) { - return NewLineIdentifier::MIX; - } if (StringUtil::Equals(value, "NOT_SET")) { return NewLineIdentifier::NOT_SET; } @@ -3822,6 +3940,8 @@ const char* EnumUtil::ToChars(ParseInfoType value) { return "COPY_INFO"; case ParseInfoType::CREATE_INFO: return "CREATE_INFO"; + case ParseInfoType::CREATE_SECRET_INFO: + return "CREATE_SECRET_INFO"; case ParseInfoType::DETACH_INFO: return "DETACH_INFO"; case ParseInfoType::DROP_INFO: @@ -3838,6 +3958,8 @@ const char* EnumUtil::ToChars(ParseInfoType value) { return "TRANSACTION_INFO"; case ParseInfoType::VACUUM_INFO: return "VACUUM_INFO"; + case ParseInfoType::COMMENT_ON_INFO: + return "COMMENT_ON_INFO"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -3857,6 +3979,9 @@ ParseInfoType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "CREATE_INFO")) { return ParseInfoType::CREATE_INFO; } + if (StringUtil::Equals(value, "CREATE_SECRET_INFO")) { + return ParseInfoType::CREATE_SECRET_INFO; + } if (StringUtil::Equals(value, "DETACH_INFO")) { return ParseInfoType::DETACH_INFO; } @@ -3881,6 +4006,9 @@ ParseInfoType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "VACUUM_INFO")) { return ParseInfoType::VACUUM_INFO; } + if (StringUtil::Equals(value, "COMMENT_ON_INFO")) { + return ParseInfoType::COMMENT_ON_INFO; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -3912,34 +4040,6 @@ ParserExtensionResultType EnumUtil::FromString(const throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } -template<> -const char* EnumUtil::ToChars(ParserMode value) { - switch(value) { - case ParserMode::PARSING: - return "PARSING"; - case ParserMode::SNIFFING_DATATYPES: - return "SNIFFING_DATATYPES"; - case ParserMode::PARSING_HEADER: - return "PARSING_HEADER"; - default: - throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); - } -} - -template<> -ParserMode EnumUtil::FromString(const char *value) { - if (StringUtil::Equals(value, "PARSING")) { - return ParserMode::PARSING; - } - if (StringUtil::Equals(value, "SNIFFING_DATATYPES")) { - return ParserMode::SNIFFING_DATATYPES; - } - if (StringUtil::Equals(value, "PARSING_HEADER")) { - return ParserMode::PARSING_HEADER; - } - throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); -} - template<> const char* EnumUtil::ToChars(PartitionSortStage value) { switch(value) { @@ -4137,8 +4237,10 @@ const char* EnumUtil::ToChars(PhysicalOperatorType value) return "PIECEWISE_MERGE_JOIN"; case PhysicalOperatorType::IE_JOIN: return "IE_JOIN"; - case PhysicalOperatorType::DELIM_JOIN: - return "DELIM_JOIN"; + case PhysicalOperatorType::LEFT_DELIM_JOIN: + return "LEFT_DELIM_JOIN"; + case PhysicalOperatorType::RIGHT_DELIM_JOIN: + return "RIGHT_DELIM_JOIN"; case PhysicalOperatorType::POSITIONAL_JOIN: return "POSITIONAL_JOIN"; case PhysicalOperatorType::ASOF_JOIN: @@ -4213,6 +4315,8 @@ const char* EnumUtil::ToChars(PhysicalOperatorType value) return "RESET"; case PhysicalOperatorType::EXTENSION: return "EXTENSION"; + case PhysicalOperatorType::CREATE_SECRET: + return "CREATE_SECRET"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -4328,8 +4432,11 @@ PhysicalOperatorType EnumUtil::FromString(const char *valu if (StringUtil::Equals(value, "IE_JOIN")) { return PhysicalOperatorType::IE_JOIN; } - if (StringUtil::Equals(value, "DELIM_JOIN")) { - return PhysicalOperatorType::DELIM_JOIN; + if (StringUtil::Equals(value, "LEFT_DELIM_JOIN")) { + return PhysicalOperatorType::LEFT_DELIM_JOIN; + } + if (StringUtil::Equals(value, "RIGHT_DELIM_JOIN")) { + return PhysicalOperatorType::RIGHT_DELIM_JOIN; } if (StringUtil::Equals(value, "POSITIONAL_JOIN")) { return PhysicalOperatorType::POSITIONAL_JOIN; @@ -4442,6 +4549,9 @@ PhysicalOperatorType EnumUtil::FromString(const char *valu if (StringUtil::Equals(value, "EXTENSION")) { return PhysicalOperatorType::EXTENSION; } + if (StringUtil::Equals(value, "CREATE_SECRET")) { + return PhysicalOperatorType::CREATE_SECRET; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -4480,6 +4590,8 @@ const char* EnumUtil::ToChars(PhysicalType value) { return "ARRAY"; case PhysicalType::VARCHAR: return "VARCHAR"; + case PhysicalType::UINT128: + return "UINT128"; case PhysicalType::INT128: return "INT128"; case PhysicalType::UNKNOWN: @@ -4543,6 +4655,9 @@ PhysicalType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "VARCHAR")) { return PhysicalType::VARCHAR; } + if (StringUtil::Equals(value, "UINT128")) { + return PhysicalType::UINT128; + } if (StringUtil::Equals(value, "INT128")) { return PhysicalType::INT128; } @@ -4996,6 +5111,57 @@ SampleMethod EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(SecretDisplayType value) { + switch(value) { + case SecretDisplayType::REDACTED: + return "REDACTED"; + case SecretDisplayType::UNREDACTED: + return "UNREDACTED"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +SecretDisplayType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "REDACTED")) { + return SecretDisplayType::REDACTED; + } + if (StringUtil::Equals(value, "UNREDACTED")) { + return SecretDisplayType::UNREDACTED; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + +template<> +const char* EnumUtil::ToChars(SecretPersistType value) { + switch(value) { + case SecretPersistType::DEFAULT: + return "DEFAULT"; + case SecretPersistType::TEMPORARY: + return "TEMPORARY"; + case SecretPersistType::PERSISTENT: + return "PERSISTENT"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +SecretPersistType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "DEFAULT")) { + return SecretPersistType::DEFAULT; + } + if (StringUtil::Equals(value, "TEMPORARY")) { + return SecretPersistType::TEMPORARY; + } + if (StringUtil::Equals(value, "PERSISTENT")) { + return SecretPersistType::PERSISTENT; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(SequenceInfo value) { switch(value) { @@ -5133,6 +5299,29 @@ SetType EnumUtil::FromString(const char *value) { throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } +template<> +const char* EnumUtil::ToChars(ShowType value) { + switch(value) { + case ShowType::SUMMARY: + return "SUMMARY"; + case ShowType::DESCRIBE: + return "DESCRIBE"; + default: + throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); + } +} + +template<> +ShowType EnumUtil::FromString(const char *value) { + if (StringUtil::Equals(value, "SUMMARY")) { + return ShowType::SUMMARY; + } + if (StringUtil::Equals(value, "DESCRIBE")) { + return ShowType::DESCRIBE; + } + throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); +} + template<> const char* EnumUtil::ToChars(SimplifiedTokenType value) { switch(value) { @@ -5373,8 +5562,6 @@ const char* EnumUtil::ToChars(StatementType value) { return "EXPORT_STATEMENT"; case StatementType::PRAGMA_STATEMENT: return "PRAGMA_STATEMENT"; - case StatementType::SHOW_STATEMENT: - return "SHOW_STATEMENT"; case StatementType::VACUUM_STATEMENT: return "VACUUM_STATEMENT"; case StatementType::CALL_STATEMENT: @@ -5458,9 +5645,6 @@ StatementType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "PRAGMA_STATEMENT")) { return StatementType::PRAGMA_STATEMENT; } - if (StringUtil::Equals(value, "SHOW_STATEMENT")) { - return StatementType::SHOW_STATEMENT; - } if (StringUtil::Equals(value, "VACUUM_STATEMENT")) { return StatementType::VACUUM_STATEMENT; } @@ -5845,6 +6029,8 @@ const char* EnumUtil::ToChars(TableFilterType value) { return "CONJUNCTION_OR"; case TableFilterType::CONJUNCTION_AND: return "CONJUNCTION_AND"; + case TableFilterType::STRUCT_EXTRACT: + return "STRUCT_EXTRACT"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -5867,6 +6053,9 @@ TableFilterType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "CONJUNCTION_AND")) { return TableFilterType::CONJUNCTION_AND; } + if (StringUtil::Equals(value, "STRUCT_EXTRACT")) { + return TableFilterType::STRUCT_EXTRACT; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -5887,10 +6076,12 @@ const char* EnumUtil::ToChars(TableReferenceType value) { return "EXPRESSION_LIST"; case TableReferenceType::CTE: return "CTE"; - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: return "EMPTY"; case TableReferenceType::PIVOT: return "PIVOT"; + case TableReferenceType::SHOW_REF: + return "SHOW_REF"; default: throw NotImplementedException(StringUtil::Format("Enum value: '%d' not implemented", value)); } @@ -5920,11 +6111,14 @@ TableReferenceType EnumUtil::FromString(const char *value) { return TableReferenceType::CTE; } if (StringUtil::Equals(value, "EMPTY")) { - return TableReferenceType::EMPTY; + return TableReferenceType::EMPTY_FROM; } if (StringUtil::Equals(value, "PIVOT")) { return TableReferenceType::PIVOT; } + if (StringUtil::Equals(value, "SHOW_REF")) { + return TableReferenceType::SHOW_REF; + } throw NotImplementedException(StringUtil::Format("Enum value: '%s' not implemented", value)); } @@ -6335,6 +6529,8 @@ const char* EnumUtil::ToChars(VerificationType value) { return "PREPARED"; case VerificationType::EXTERNAL: return "EXTERNAL"; + case VerificationType::FETCH_ROW_AS_SCAN: + return "FETCH_ROW_AS_SCAN"; case VerificationType::INVALID: return "INVALID"; default: @@ -6368,6 +6564,9 @@ VerificationType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "EXTERNAL")) { return VerificationType::EXTERNAL; } + if (StringUtil::Equals(value, "FETCH_ROW_AS_SCAN")) { + return VerificationType::FETCH_ROW_AS_SCAN; + } if (StringUtil::Equals(value, "INVALID")) { return VerificationType::INVALID; } @@ -6451,6 +6650,8 @@ const char* EnumUtil::ToChars(WALType value) { return "DELETE_TUPLE"; case WALType::UPDATE_TUPLE: return "UPDATE_TUPLE"; + case WALType::WAL_VERSION: + return "WAL_VERSION"; case WALType::CHECKPOINT: return "CHECKPOINT"; case WALType::WAL_FLUSH: @@ -6531,6 +6732,9 @@ WALType EnumUtil::FromString(const char *value) { if (StringUtil::Equals(value, "UPDATE_TUPLE")) { return WALType::UPDATE_TUPLE; } + if (StringUtil::Equals(value, "WAL_VERSION")) { + return WALType::WAL_VERSION; + } if (StringUtil::Equals(value, "CHECKPOINT")) { return WALType::CHECKPOINT; } diff --git a/src/duckdb/src/common/enums/catalog_type.cpp b/src/duckdb/src/common/enums/catalog_type.cpp index 3a7b285f7..55dbcb41a 100644 --- a/src/duckdb/src/common/enums/catalog_type.cpp +++ b/src/duckdb/src/common/enums/catalog_type.cpp @@ -39,6 +39,12 @@ string CatalogTypeToString(CatalogType type) { return "Prepared Statement"; case CatalogType::SEQUENCE_ENTRY: return "Sequence"; + case CatalogType::SECRET_ENTRY: + return "Secret"; + case CatalogType::SECRET_TYPE_ENTRY: + return "Secret Type"; + case CatalogType::SECRET_FUNCTION_ENTRY: + return "Secret Function"; case CatalogType::INVALID: case CatalogType::DELETED_ENTRY: case CatalogType::RENAMED_ENTRY: diff --git a/src/duckdb/src/common/enums/compression_type.cpp b/src/duckdb/src/common/enums/compression_type.cpp index 267c2253e..3b9930cfa 100644 --- a/src/duckdb/src/common/enums/compression_type.cpp +++ b/src/duckdb/src/common/enums/compression_type.cpp @@ -16,6 +16,12 @@ vector ListCompressionTypes(void) { return compression_types; } +bool CompressionTypeIsDeprecated(CompressionType compression_type) { + const bool is_patas = compression_type == CompressionType::COMPRESSION_PATAS; + const bool is_chimp = compression_type == CompressionType::COMPRESSION_CHIMP; + return (is_patas || is_chimp); +} + CompressionType CompressionTypeFromString(const string &str) { auto compression = StringUtil::Lower(str); if (compression == "uncompressed") { @@ -34,6 +40,10 @@ CompressionType CompressionTypeFromString(const string &str) { return CompressionType::COMPRESSION_CHIMP; } else if (compression == "patas") { return CompressionType::COMPRESSION_PATAS; + } else if (compression == "alp") { + return CompressionType::COMPRESSION_ALP; + } else if (compression == "alprd") { + return CompressionType::COMPRESSION_ALPRD; } else { return CompressionType::COMPRESSION_AUTO; } @@ -61,6 +71,10 @@ string CompressionTypeToString(CompressionType type) { return "Chimp"; case CompressionType::COMPRESSION_PATAS: return "Patas"; + case CompressionType::COMPRESSION_ALP: + return "ALP"; + case CompressionType::COMPRESSION_ALPRD: + return "ALPRD"; default: throw InternalException("Unrecognized compression type!"); } diff --git a/src/duckdb/src/common/enums/date_part_specifier.cpp b/src/duckdb/src/common/enums/date_part_specifier.cpp index 4a633a515..032a82165 100644 --- a/src/duckdb/src/common/enums/date_part_specifier.cpp +++ b/src/duckdb/src/common/enums/date_part_specifier.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/enums/date_part_specifier.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/common/enums/file_compression_type.cpp b/src/duckdb/src/common/enums/file_compression_type.cpp index c3c790d23..5df6add19 100644 --- a/src/duckdb/src/common/enums/file_compression_type.cpp +++ b/src/duckdb/src/common/enums/file_compression_type.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/enums/file_compression_type.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/exception/parser_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/common/enums/join_type.cpp b/src/duckdb/src/common/enums/join_type.cpp index 7d7e96c52..6bfe54892 100644 --- a/src/duckdb/src/common/enums/join_type.cpp +++ b/src/duckdb/src/common/enums/join_type.cpp @@ -16,6 +16,34 @@ bool PropagatesBuildSide(JoinType type) { type == JoinType::RIGHT_SEMI; } +bool HasInverseJoinType(JoinType type) { + return type != JoinType::SINGLE && type != JoinType::MARK; +} + +JoinType InverseJoinType(JoinType type) { + D_ASSERT(HasInverseJoinType(type)); + switch (type) { + case JoinType::LEFT: + return JoinType::RIGHT; + case JoinType::RIGHT: + return JoinType::LEFT; + case JoinType::INNER: + return JoinType::INNER; + case JoinType::OUTER: + return JoinType::OUTER; + case JoinType::SEMI: + return JoinType::RIGHT_SEMI; + case JoinType::ANTI: + return JoinType::RIGHT_ANTI; + case JoinType::RIGHT_SEMI: + return JoinType::SEMI; + case JoinType::RIGHT_ANTI: + return JoinType::ANTI; + default: + throw NotImplementedException("InverseJoinType for JoinType::%s", EnumUtil::ToString(type)); + } +} + // **DEPRECATED**: Use EnumUtil directly instead. string JoinTypeToString(JoinType type) { return EnumUtil::ToString(type); diff --git a/src/duckdb/src/common/enums/logical_operator_type.cpp b/src/duckdb/src/common/enums/logical_operator_type.cpp index 246d1256e..495bf59d5 100644 --- a/src/duckdb/src/common/enums/logical_operator_type.cpp +++ b/src/duckdb/src/common/enums/logical_operator_type.cpp @@ -94,8 +94,6 @@ string LogicalOperatorToString(LogicalOperatorType type) { return "CTE"; case LogicalOperatorType::LOGICAL_CTE_REF: return "CTE_SCAN"; - case LogicalOperatorType::LOGICAL_SHOW: - return "SHOW"; case LogicalOperatorType::LOGICAL_ALTER: return "ALTER"; case LogicalOperatorType::LOGICAL_CREATE_SEQUENCE: @@ -106,10 +104,12 @@ string LogicalOperatorToString(LogicalOperatorType type) { return "CREATE_VIEW"; case LogicalOperatorType::LOGICAL_CREATE_SCHEMA: return "CREATE_SCHEMA"; + case LogicalOperatorType::LOGICAL_CREATE_SECRET: + return "CREATE_SECRET"; case LogicalOperatorType::LOGICAL_ATTACH: return "ATTACH"; case LogicalOperatorType::LOGICAL_DETACH: - return "ATTACH"; + return "DETACH"; case LogicalOperatorType::LOGICAL_DROP: return "DROP"; case LogicalOperatorType::LOGICAL_PRAGMA: diff --git a/src/duckdb/src/common/enums/optimizer_type.cpp b/src/duckdb/src/common/enums/optimizer_type.cpp index a8365ab76..828d7eac8 100644 --- a/src/duckdb/src/common/enums/optimizer_type.cpp +++ b/src/duckdb/src/common/enums/optimizer_type.cpp @@ -1,6 +1,6 @@ #include "duckdb/common/enums/optimizer_type.hpp" #include "duckdb/common/string_util.hpp" - +#include "duckdb/common/exception/parser_exception.hpp" #include "duckdb/common/exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/common/enums/physical_operator_type.cpp b/src/duckdb/src/common/enums/physical_operator_type.cpp index f00b0356c..54b6fc7ac 100644 --- a/src/duckdb/src/common/enums/physical_operator_type.cpp +++ b/src/duckdb/src/common/enums/physical_operator_type.cpp @@ -51,8 +51,10 @@ string PhysicalOperatorToString(PhysicalOperatorType type) { return "BATCH_COPY_TO_FILE"; case PhysicalOperatorType::FIXED_BATCH_COPY_TO_FILE: return "FIXED_BATCH_COPY_TO_FILE"; - case PhysicalOperatorType::DELIM_JOIN: - return "DELIM_JOIN"; + case PhysicalOperatorType::LEFT_DELIM_JOIN: + return "LEFT_DELIM_JOIN"; + case PhysicalOperatorType::RIGHT_DELIM_JOIN: + return "RIGHT_DELIM_JOIN"; case PhysicalOperatorType::BLOCKWISE_NL_JOIN: return "BLOCKWISE_NL_JOIN"; case PhysicalOperatorType::NESTED_LOOP_JOIN: @@ -119,6 +121,8 @@ string PhysicalOperatorToString(PhysicalOperatorType type) { return "CREATE_SCHEMA"; case PhysicalOperatorType::CREATE_MACRO: return "CREATE_MACRO"; + case PhysicalOperatorType::CREATE_SECRET: + return "CREATE_SECRET"; case PhysicalOperatorType::DROP: return "DROP"; case PhysicalOperatorType::PRAGMA: diff --git a/src/duckdb/src/common/enums/statement_type.cpp b/src/duckdb/src/common/enums/statement_type.cpp index 2af4371ad..eddee179e 100644 --- a/src/duckdb/src/common/enums/statement_type.cpp +++ b/src/duckdb/src/common/enums/statement_type.cpp @@ -39,8 +39,6 @@ string StatementTypeToString(StatementType type) { return "DROP"; case StatementType::PRAGMA_STATEMENT: return "PRAGMA"; - case StatementType::SHOW_STATEMENT: - return "SHOW"; case StatementType::VACUUM_STATEMENT: return "VACUUM"; case StatementType::RELATION_STATEMENT: diff --git a/src/duckdb/src/common/error_data.cpp b/src/duckdb/src/common/error_data.cpp new file mode 100644 index 000000000..add9a273d --- /dev/null +++ b/src/duckdb/src/common/error_data.cpp @@ -0,0 +1,113 @@ +#include "duckdb/common/error_data.hpp" +#include "duckdb/common/exception.hpp" + +#include "duckdb/common/string_util.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/common/types.hpp" +#include "duckdb/parser/parsed_expression.hpp" +#include "duckdb/parser/query_error_context.hpp" +#include "duckdb/parser/tableref.hpp" + +namespace duckdb { + +ErrorData::ErrorData() : initialized(false), type(ExceptionType::INVALID) { +} + +ErrorData::ErrorData(const std::exception &ex) : ErrorData(ex.what()) { +} + +ErrorData::ErrorData(ExceptionType type, const string &message) + : initialized(true), type(type), raw_message(SanitizeErrorMessage(message)) { +} + +ErrorData::ErrorData(const string &message) : initialized(true), type(ExceptionType::INVALID), raw_message(string()) { + + // parse the constructed JSON + if (message.empty() || message[0] != '{') { + // not JSON! Use the message as a raw Exception message and leave type as uninitialized + raw_message = message; + return; + } else { + auto info = StringUtil::ParseJSONMap(message); + for (auto &entry : info) { + if (entry.first == "exception_type") { + type = Exception::StringToExceptionType(entry.second); + } else if (entry.first == "exception_message") { + raw_message = SanitizeErrorMessage(entry.second); + } else { + extra_info[entry.first] = entry.second; + } + } + } +} + +const string &ErrorData::Message() { + if (final_message.empty()) { + final_message = Exception::ExceptionTypeToString(type) + " Error: " + raw_message; + } + return final_message; +} + +string ErrorData::SanitizeErrorMessage(string error) { + return StringUtil::Replace(std::move(error), string("\0", 1), "\\0"); +} + +void ErrorData::Throw(const string &prepended_message) const { + D_ASSERT(initialized); + if (!prepended_message.empty()) { + string new_message = prepended_message + raw_message; + throw Exception(type, new_message, extra_info); + } else { + throw Exception(type, raw_message, extra_info); + } +} + +const ExceptionType &ErrorData::Type() const { + D_ASSERT(initialized); + return this->type; +} + +bool ErrorData::operator==(const ErrorData &other) const { + if (initialized != other.initialized) { + return false; + } + if (type != other.type) { + return false; + } + return raw_message == other.raw_message; +} + +void ErrorData::ConvertErrorToJSON() { + if (raw_message.empty() || raw_message[0] == '{') { + // empty or already JSON + return; + } + raw_message = StringUtil::ToJSONMap(type, raw_message, extra_info); + final_message = raw_message; +} + +void ErrorData::AddErrorLocation(const string &query) { + auto entry = extra_info.find("position"); + if (entry == extra_info.end()) { + return; + } + raw_message = QueryErrorContext::Format(query, raw_message, std::stoull(entry->second)); +} + +void ErrorData::AddQueryLocation(optional_idx query_location) { + Exception::SetQueryLocation(query_location, extra_info); +} + +void ErrorData::AddQueryLocation(QueryErrorContext error_context) { + AddQueryLocation(error_context.query_location); +} + +void ErrorData::AddQueryLocation(const ParsedExpression &ref) { + AddQueryLocation(ref.query_location); +} + +void ErrorData::AddQueryLocation(const TableRef &ref) { + AddQueryLocation(ref.query_location); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/exception.cpp b/src/duckdb/src/common/exception.cpp index 882d47dff..ba742c0a9 100644 --- a/src/duckdb/src/common/exception.cpp +++ b/src/duckdb/src/common/exception.cpp @@ -2,6 +2,9 @@ #include "duckdb/common/string_util.hpp" #include "duckdb/common/to_string.hpp" #include "duckdb/common/types.hpp" +#include "duckdb/common/exception/list.hpp" +#include "duckdb/parser/tableref.hpp" +#include "duckdb/planner/expression.hpp" #ifdef DUCKDB_CRASH_ON_ASSERT #include "duckdb/common/printer.hpp" @@ -14,21 +17,22 @@ namespace duckdb { -Exception::Exception(const string &msg) : std::exception(), type(ExceptionType::INVALID), raw_message_(msg) { - exception_message_ = msg; +Exception::Exception(ExceptionType exception_type, const string &message) + : std::runtime_error(ToJSON(exception_type, message)) { } -Exception::Exception(ExceptionType exception_type, const string &message) - : std::exception(), type(exception_type), raw_message_(message) { - exception_message_ = ExceptionTypeToString(exception_type) + " Error: " + message; +Exception::Exception(ExceptionType exception_type, const string &message, + const unordered_map &extra_info) + : std::runtime_error(ToJSON(exception_type, message, extra_info)) { } -const char *Exception::what() const noexcept { - return exception_message_.c_str(); +string Exception::ToJSON(ExceptionType type, const string &message) { + unordered_map extra_info; + return ToJSON(type, message, extra_info); } -const string &Exception::RawMessage() const { - return raw_message_; +string Exception::ToJSON(ExceptionType type, const string &message, const unordered_map &extra_info) { + return StringUtil::ToJSONMap(type, message, extra_info); } bool Exception::UncaughtException() { @@ -39,6 +43,30 @@ bool Exception::UncaughtException() { #endif } +bool Exception::InvalidatesTransaction(ExceptionType exception_type) { + switch (exception_type) { + case ExceptionType::BINDER: + case ExceptionType::CATALOG: + case ExceptionType::CONNECTION: + case ExceptionType::PARAMETER_NOT_ALLOWED: + case ExceptionType::PARSER: + case ExceptionType::PERMISSION: + return false; + default: + return true; + } +} + +bool Exception::InvalidatesDatabase(ExceptionType exception_type) { + switch (exception_type) { + case ExceptionType::INTERNAL: + case ExceptionType::FATAL: + return true; + default: + return false; + } +} + string Exception::GetStackTrace(int max_depth) { #ifdef DUCKDB_DEBUG_STACKTRACE string result; @@ -125,7 +153,8 @@ static constexpr ExceptionEntry EXCEPTION_MAP[] = {{ExceptionType::INVALID, "Inv {ExceptionType::DEPENDENCY, "Dependency"}, {ExceptionType::MISSING_EXTENSION, "Missing Extension"}, {ExceptionType::HTTP, "HTTP"}, - {ExceptionType::AUTOLOAD, "Extension Autoloading"}}; + {ExceptionType::AUTOLOAD, "Extension Autoloading"}, + {ExceptionType::SEQUENCE, "Sequence"}}; string Exception::ExceptionTypeToString(ExceptionType type) { for (auto &e : EXCEPTION_MAP) { @@ -145,121 +174,49 @@ ExceptionType Exception::StringToExceptionType(const string &type) { return ExceptionType::INVALID; } -const HTTPException &Exception::AsHTTPException() const { - D_ASSERT(type == ExceptionType::HTTP); - const auto &e = static_cast(this); - D_ASSERT(e->GetStatusCode() != 0); - D_ASSERT(e->GetHeaders().size() > 0); - return *e; -} - -void Exception::ThrowAsTypeWithMessage(ExceptionType type, const string &message, - const std::shared_ptr &original) { - switch (type) { - case ExceptionType::OUT_OF_RANGE: - throw OutOfRangeException(message); - case ExceptionType::CONVERSION: - throw ConversionException(message); // FIXME: make a separation between Conversion/Cast exception? - case ExceptionType::INVALID_TYPE: - throw InvalidTypeException(message); - case ExceptionType::MISMATCH_TYPE: - throw TypeMismatchException(message); - case ExceptionType::TRANSACTION: - throw TransactionException(message); - case ExceptionType::NOT_IMPLEMENTED: - throw NotImplementedException(message); - case ExceptionType::CATALOG: - throw CatalogException(message); - case ExceptionType::CONNECTION: - throw ConnectionException(message); - case ExceptionType::PARSER: - throw ParserException(message); - case ExceptionType::PERMISSION: - throw PermissionException(message); - case ExceptionType::SYNTAX: - throw SyntaxException(message); - case ExceptionType::CONSTRAINT: - throw ConstraintException(message); - case ExceptionType::BINDER: - throw BinderException(message); - case ExceptionType::IO: - throw IOException(message); - case ExceptionType::SERIALIZATION: - throw SerializationException(message); - case ExceptionType::INTERRUPT: - throw InterruptException(); - case ExceptionType::INTERNAL: - throw InternalException(message); - case ExceptionType::INVALID_INPUT: - throw InvalidInputException(message); - case ExceptionType::OUT_OF_MEMORY: - throw OutOfMemoryException(message); - case ExceptionType::PARAMETER_NOT_ALLOWED: - throw ParameterNotAllowedException(message); - case ExceptionType::PARAMETER_NOT_RESOLVED: - throw ParameterNotResolvedException(); - case ExceptionType::FATAL: - throw FatalException(message); - case ExceptionType::DEPENDENCY: - throw DependencyException(message); - case ExceptionType::HTTP: { - original->AsHTTPException().Throw(); - } - case ExceptionType::MISSING_EXTENSION: - throw MissingExtensionException(message); - default: - throw Exception(type, message); - } -} - -StandardException::StandardException(ExceptionType exception_type, const string &message) - : Exception(exception_type, message) { +unordered_map Exception::InitializeExtraInfo(const Expression &expr) { + return InitializeExtraInfo(expr.query_location); } -CastException::CastException(const PhysicalType orig_type, const PhysicalType new_type) - : Exception(ExceptionType::CONVERSION, - "Type " + TypeIdToString(orig_type) + " can't be cast as " + TypeIdToString(new_type)) { +unordered_map Exception::InitializeExtraInfo(const ParsedExpression &expr) { + return InitializeExtraInfo(expr.query_location); } -CastException::CastException(const LogicalType &orig_type, const LogicalType &new_type) - : Exception(ExceptionType::CONVERSION, - "Type " + orig_type.ToString() + " can't be cast as " + new_type.ToString()) { +unordered_map Exception::InitializeExtraInfo(const QueryErrorContext &error_context) { + return InitializeExtraInfo(error_context.query_location); } -CastException::CastException(const string &msg) : Exception(ExceptionType::CONVERSION, msg) { +unordered_map Exception::InitializeExtraInfo(const TableRef &ref) { + return InitializeExtraInfo(ref.query_location); } -ValueOutOfRangeException::ValueOutOfRangeException(const int64_t value, const PhysicalType orig_type, - const PhysicalType new_type) - : Exception(ExceptionType::CONVERSION, "Type " + TypeIdToString(orig_type) + " with value " + - to_string((intmax_t)value) + - " can't be cast because the value is out of range " - "for the destination type " + - TypeIdToString(new_type)) { +unordered_map Exception::InitializeExtraInfo(optional_idx error_location) { + unordered_map result; + SetQueryLocation(error_location, result); + return result; } -ValueOutOfRangeException::ValueOutOfRangeException(const double value, const PhysicalType orig_type, - const PhysicalType new_type) - : Exception(ExceptionType::CONVERSION, "Type " + TypeIdToString(orig_type) + " with value " + to_string(value) + - " can't be cast because the value is out of range " - "for the destination type " + - TypeIdToString(new_type)) { +unordered_map Exception::InitializeExtraInfo(const string &subtype, optional_idx error_location) { + unordered_map result; + result["error_subtype"] = subtype; + SetQueryLocation(error_location, result); + return result; } -ValueOutOfRangeException::ValueOutOfRangeException(const hugeint_t value, const PhysicalType orig_type, - const PhysicalType new_type) - : Exception(ExceptionType::CONVERSION, "Type " + TypeIdToString(orig_type) + " with value " + value.ToString() + - " can't be cast because the value is out of range " - "for the destination type " + - TypeIdToString(new_type)) { +void Exception::SetQueryLocation(optional_idx error_location, unordered_map &extra_info) { + if (error_location.IsValid()) { + extra_info["position"] = to_string(error_location.GetIndex()); + } } -ValueOutOfRangeException::ValueOutOfRangeException(const PhysicalType var_type, const idx_t length) - : Exception(ExceptionType::OUT_OF_RANGE, - "The value is too long to fit into type " + TypeIdToString(var_type) + "(" + to_string(length) + ")") { +ConversionException::ConversionException(const PhysicalType orig_type, const PhysicalType new_type) + : Exception(ExceptionType::CONVERSION, + "Type " + TypeIdToString(orig_type) + " can't be cast as " + TypeIdToString(new_type)) { } -ValueOutOfRangeException::ValueOutOfRangeException(const string &msg) : Exception(ExceptionType::OUT_OF_RANGE, msg) { +ConversionException::ConversionException(const LogicalType &orig_type, const LogicalType &new_type) + : Exception(ExceptionType::CONVERSION, + "Type " + orig_type.ToString() + " can't be cast as " + new_type.ToString()) { } ConversionException::ConversionException(const string &msg) : Exception(ExceptionType::CONVERSION, msg) { @@ -298,16 +255,38 @@ NotImplementedException::NotImplementedException(const string &msg) : Exception( OutOfRangeException::OutOfRangeException(const string &msg) : Exception(ExceptionType::OUT_OF_RANGE, msg) { } -CatalogException::CatalogException(const string &msg) : StandardException(ExceptionType::CATALOG, msg) { +OutOfRangeException::OutOfRangeException(const int64_t value, const PhysicalType orig_type, const PhysicalType new_type) + : Exception(ExceptionType::OUT_OF_RANGE, "Type " + TypeIdToString(orig_type) + " with value " + + to_string((intmax_t)value) + + " can't be cast because the value is out of range " + "for the destination type " + + TypeIdToString(new_type)) { } -ConnectionException::ConnectionException(const string &msg) : StandardException(ExceptionType::CONNECTION, msg) { +OutOfRangeException::OutOfRangeException(const double value, const PhysicalType orig_type, const PhysicalType new_type) + : Exception(ExceptionType::OUT_OF_RANGE, "Type " + TypeIdToString(orig_type) + " with value " + to_string(value) + + " can't be cast because the value is out of range " + "for the destination type " + + TypeIdToString(new_type)) { } -ParserException::ParserException(const string &msg) : StandardException(ExceptionType::PARSER, msg) { +OutOfRangeException::OutOfRangeException(const hugeint_t value, const PhysicalType orig_type, + const PhysicalType new_type) + : Exception(ExceptionType::OUT_OF_RANGE, "Type " + TypeIdToString(orig_type) + " with value " + value.ToString() + + " can't be cast because the value is out of range " + "for the destination type " + + TypeIdToString(new_type)) { +} + +OutOfRangeException::OutOfRangeException(const PhysicalType var_type, const idx_t length) + : Exception(ExceptionType::OUT_OF_RANGE, + "The value is too long to fit into type " + TypeIdToString(var_type) + "(" + to_string(length) + ")") { } -PermissionException::PermissionException(const string &msg) : StandardException(ExceptionType::PERMISSION, msg) { +ConnectionException::ConnectionException(const string &msg) : Exception(ExceptionType::CONNECTION, msg) { +} + +PermissionException::PermissionException(const string &msg) : Exception(ExceptionType::PERMISSION, msg) { } SyntaxException::SyntaxException(const string &msg) : Exception(ExceptionType::SYNTAX, msg) { @@ -319,9 +298,6 @@ ConstraintException::ConstraintException(const string &msg) : Exception(Exceptio DependencyException::DependencyException(const string &msg) : Exception(ExceptionType::DEPENDENCY, msg) { } -BinderException::BinderException(const string &msg) : StandardException(ExceptionType::BINDER, msg) { -} - IOException::IOException(const string &msg) : Exception(ExceptionType::IO, msg) { } @@ -329,17 +305,16 @@ MissingExtensionException::MissingExtensionException(const string &msg) : Exception(ExceptionType::MISSING_EXTENSION, msg) { } -AutoloadException::AutoloadException(const string &extension_name, Exception &e) +AutoloadException::AutoloadException(const string &extension_name, const string &message) : Exception(ExceptionType::AUTOLOAD, "An error occurred while trying to automatically install the required extension '" + extension_name + - "':\n" + e.RawMessage()), - wrapped_exception(e) { + "':\n" + message) { } SerializationException::SerializationException(const string &msg) : Exception(ExceptionType::SERIALIZATION, msg) { } -SequenceException::SequenceException(const string &msg) : Exception(ExceptionType::SERIALIZATION, msg) { +SequenceException::SequenceException(const string &msg) : Exception(ExceptionType::SEQUENCE, msg) { } InterruptException::InterruptException() : Exception(ExceptionType::INTERRUPT, "Interrupted!") { @@ -348,7 +323,7 @@ InterruptException::InterruptException() : Exception(ExceptionType::INTERRUPT, " FatalException::FatalException(ExceptionType type, const string &msg) : Exception(type, msg) { } -InternalException::InternalException(const string &msg) : FatalException(ExceptionType::INTERNAL, msg) { +InternalException::InternalException(const string &msg) : Exception(ExceptionType::INTERNAL, msg) { #ifdef DUCKDB_CRASH_ON_ASSERT Printer::Print("ABORT THROWN BY INTERNAL EXCEPTION: " + msg); abort(); @@ -358,11 +333,15 @@ InternalException::InternalException(const string &msg) : FatalException(Excepti InvalidInputException::InvalidInputException(const string &msg) : Exception(ExceptionType::INVALID_INPUT, msg) { } +InvalidInputException::InvalidInputException(const string &msg, const unordered_map &extra_info) + : Exception(ExceptionType::INVALID_INPUT, msg, extra_info) { +} + OutOfMemoryException::OutOfMemoryException(const string &msg) : Exception(ExceptionType::OUT_OF_MEMORY, msg) { } ParameterNotAllowedException::ParameterNotAllowedException(const string &msg) - : StandardException(ExceptionType::PARAMETER_NOT_ALLOWED, msg) { + : Exception(ExceptionType::PARAMETER_NOT_ALLOWED, msg) { } ParameterNotResolvedException::ParameterNotResolvedException() diff --git a/src/duckdb/src/common/exception/binder_exception.cpp b/src/duckdb/src/common/exception/binder_exception.cpp new file mode 100644 index 000000000..458c563c0 --- /dev/null +++ b/src/duckdb/src/common/exception/binder_exception.cpp @@ -0,0 +1,47 @@ +#include "duckdb/common/exception/binder_exception.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/function/function.hpp" + +namespace duckdb { + +BinderException::BinderException(const string &msg) : Exception(ExceptionType::BINDER, msg) { +} + +BinderException::BinderException(const string &msg, const unordered_map &extra_info) + : Exception(ExceptionType::BINDER, msg, extra_info) { +} + +BinderException BinderException::ColumnNotFound(const string &name, const vector &similar_bindings, + QueryErrorContext context) { + auto extra_info = Exception::InitializeExtraInfo("COLUMN_NOT_FOUND", context.query_location); + string candidate_str = StringUtil::CandidatesMessage(similar_bindings, "Candidate bindings"); + extra_info["name"] = name; + if (!similar_bindings.empty()) { + extra_info["candidates"] = StringUtil::Join(similar_bindings, ","); + } + return BinderException( + StringUtil::Format("Referenced column \"%s\" not found in FROM clause!%s", name, candidate_str), extra_info); +} + +BinderException BinderException::NoMatchingFunction(const string &name, const vector &arguments, + const vector &candidates) { + auto extra_info = Exception::InitializeExtraInfo("NO_MATCHING_FUNCTION", optional_idx()); + // no matching function was found, throw an error + string call_str = Function::CallToString(name, arguments); + string candidate_str; + for (auto &candidate : candidates) { + candidate_str += "\t" + candidate + "\n"; + } + extra_info["name"] = name; + extra_info["call"] = call_str; + if (!candidates.empty()) { + extra_info["candidates"] = StringUtil::Join(candidates, ","); + } + return BinderException( + StringUtil::Format("No function matches the given name and argument types '%s'. You might need to add " + "explicit type casts.\n\tCandidate functions:\n%s", + call_str, candidate_str), + extra_info); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/exception/catalog_exception.cpp b/src/duckdb/src/common/exception/catalog_exception.cpp new file mode 100644 index 000000000..2ab210f0e --- /dev/null +++ b/src/duckdb/src/common/exception/catalog_exception.cpp @@ -0,0 +1,55 @@ +#include "duckdb/common/exception/catalog_exception.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/common/string_util.hpp" + +namespace duckdb { + +CatalogException::CatalogException(const string &msg) : Exception(ExceptionType::CATALOG, msg) { +} + +CatalogException::CatalogException(const string &msg, const unordered_map &extra_info) + : Exception(ExceptionType::CATALOG, msg, extra_info) { +} + +CatalogException CatalogException::MissingEntry(CatalogType type, const string &name, const string &suggestion, + QueryErrorContext context) { + string did_you_mean; + if (!suggestion.empty()) { + did_you_mean = "\nDid you mean \"" + suggestion + "\"?"; + } + + auto extra_info = Exception::InitializeExtraInfo("MISSING_ENTRY", context.query_location); + + extra_info["name"] = name; + extra_info["type"] = CatalogTypeToString(type); + if (!suggestion.empty()) { + extra_info["candidates"] = suggestion; + } + return CatalogException( + StringUtil::Format("%s with name %s does not exist!%s", CatalogTypeToString(type), name, did_you_mean), + extra_info); +} + +CatalogException CatalogException::MissingEntry(const string &type, const string &name, + const vector &suggestions, QueryErrorContext context) { + auto extra_info = Exception::InitializeExtraInfo("MISSING_ENTRY", context.query_location); + extra_info["error_subtype"] = "MISSING_ENTRY"; + extra_info["name"] = name; + extra_info["type"] = type; + if (!suggestions.empty()) { + extra_info["candidates"] = StringUtil::Join(suggestions, ", "); + } + return CatalogException(StringUtil::Format("unrecognized %s \"%s\"\n%s", type, name, + StringUtil::CandidatesErrorMessage(suggestions, name, "Did you mean")), + extra_info); +} + +CatalogException CatalogException::EntryAlreadyExists(CatalogType type, const string &name, QueryErrorContext context) { + auto extra_info = Exception::InitializeExtraInfo("ENTRY_ALREADY_EXISTS", optional_idx()); + extra_info["name"] = name; + extra_info["type"] = CatalogTypeToString(type); + return CatalogException(StringUtil::Format("%s with name \"%s\" already exists!", CatalogTypeToString(type), name), + extra_info); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/exception/parser_exception.cpp b/src/duckdb/src/common/exception/parser_exception.cpp new file mode 100644 index 000000000..f3875da38 --- /dev/null +++ b/src/duckdb/src/common/exception/parser_exception.cpp @@ -0,0 +1,19 @@ +#include "duckdb/common/exception/parser_exception.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/parser/query_error_context.hpp" + +namespace duckdb { + +ParserException::ParserException(const string &msg) : Exception(ExceptionType::PARSER, msg) { +} + +ParserException::ParserException(const string &msg, const unordered_map &extra_info) + : Exception(ExceptionType::PARSER, msg, extra_info) { +} + +ParserException ParserException::SyntaxError(const string &query, const string &error_message, + optional_idx error_location) { + return ParserException(error_message, Exception::InitializeExtraInfo("SYNTAX_ERROR", error_location)); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/exception_format_value.cpp b/src/duckdb/src/common/exception_format_value.cpp index 5a4330302..1eb9d4593 100644 --- a/src/duckdb/src/common/exception_format_value.cpp +++ b/src/duckdb/src/common/exception_format_value.cpp @@ -3,6 +3,7 @@ #include "fmt/format.h" #include "fmt/printf.h" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/parser/keyword_helper.hpp" namespace duckdb { @@ -16,6 +17,9 @@ ExceptionFormatValue::ExceptionFormatValue(int64_t int_val) ExceptionFormatValue::ExceptionFormatValue(hugeint_t huge_val) : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_STRING), str_val(Hugeint::ToString(huge_val)) { } +ExceptionFormatValue::ExceptionFormatValue(uhugeint_t uhuge_val) + : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_STRING), str_val(Uhugeint::ToString(uhuge_val)) { +} ExceptionFormatValue::ExceptionFormatValue(string str_val) : type(ExceptionFormatValueType::FORMAT_VALUE_TYPE_STRING), str_val(std::move(str_val)) { } @@ -66,6 +70,10 @@ template <> ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(hugeint_t value) { return ExceptionFormatValue(value); } +template <> +ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(uhugeint_t value) { + return ExceptionFormatValue(value); +} string ExceptionFormatValue::Format(const string &msg, std::vector &values) { try { @@ -88,7 +96,7 @@ string ExceptionFormatValue::Format(const string &msg, std::vectorCast(); + return target_type == other.target_type && cast_score == other.cast_score; +} + +//===--------------------------------------------------------------------===// +// Any Type Info +//===--------------------------------------------------------------------===// +IntegerLiteralTypeInfo::IntegerLiteralTypeInfo() : ExtraTypeInfo(ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO) { +} + +IntegerLiteralTypeInfo::IntegerLiteralTypeInfo(Value constant_value_p) + : ExtraTypeInfo(ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO), constant_value(std::move(constant_value_p)) { +} + +bool IntegerLiteralTypeInfo::EqualsInternal(ExtraTypeInfo *other_p) const { + auto &other = other_p->Cast(); + return constant_value == other.constant_value; +} + } // namespace duckdb diff --git a/src/duckdb/src/common/file_system.cpp b/src/duckdb/src/common/file_system.cpp index e110a5477..f1a72e0b8 100644 --- a/src/duckdb/src/common/file_system.cpp +++ b/src/duckdb/src/common/file_system.cpp @@ -544,7 +544,7 @@ FileType FileHandle::GetType() { } bool FileSystem::IsRemoteFile(const string &path) { - const string prefixes[] = {"http://", "https://", "s3://", "s3a://", "s3n://", "gcs://", "r2://"}; + const string prefixes[] = {"http://", "https://", "s3://", "s3a://", "s3n://", "gcs://", "gs://", "r2://"}; for (auto &prefix : prefixes) { if (StringUtil::StartsWith(path, prefix)) { return true; diff --git a/src/duckdb/src/common/hive_partitioning.cpp b/src/duckdb/src/common/hive_partitioning.cpp index 5a90a2935..0bb3306b7 100644 --- a/src/duckdb/src/common/hive_partitioning.cpp +++ b/src/duckdb/src/common/hive_partitioning.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/hive_partitioning.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/optimizer/filter_combiner.hpp" #include "duckdb/planner/expression/bound_columnref_expression.hpp" @@ -261,6 +262,9 @@ static void GetHivePartitionValuesTypeSwitch(Vector &input, vector(input, keys, col_idx, count); break; + case PhysicalType::UINT128: + TemplatedGetHivePartitionValues(input, keys, col_idx, count); + break; case PhysicalType::FLOAT: TemplatedGetHivePartitionValues(input, keys, col_idx, count); break; diff --git a/src/duckdb/src/common/http_state.cpp b/src/duckdb/src/common/http_state.cpp index 4a50ecd70..0c6920867 100644 --- a/src/duckdb/src/common/http_state.cpp +++ b/src/duckdb/src/common/http_state.cpp @@ -10,13 +10,14 @@ CachedFileHandle::CachedFileHandle(shared_ptr &file_p) { file = file_p; } -void CachedFileHandle::SetInitialized() { +void CachedFileHandle::SetInitialized(idx_t total_size) { if (file->initialized) { throw InternalException("Cannot set initialized on cached file that was already initialized"); } if (!lock) { throw InternalException("Cannot set initialized on cached file without lock"); } + file->size = total_size; file->initialized = true; lock = nullptr; } @@ -57,10 +58,26 @@ void HTTPState::Reset() { cached_files.clear(); } -shared_ptr HTTPState::TryGetState(FileOpener *opener) { +shared_ptr HTTPState::TryGetState(ClientContext &context, bool create_on_missing) { + auto lookup = context.registered_state.find("http_state"); + + if (lookup != context.registered_state.end()) { + return std::static_pointer_cast(lookup->second); + } + + if (!create_on_missing) { + return nullptr; + } + + auto http_state = make_shared(); + context.registered_state["http_state"] = http_state; + return http_state; +} + +shared_ptr HTTPState::TryGetState(FileOpener *opener, bool create_on_missing) { auto client_context = FileOpener::TryGetClientContext(opener); if (client_context) { - return client_context->client_data->http_state; + return TryGetState(*client_context, create_on_missing); } return nullptr; } diff --git a/src/duckdb/src/common/local_file_system.cpp b/src/duckdb/src/common/local_file_system.cpp index ccf3e1cb0..347a21f19 100644 --- a/src/duckdb/src/common/local_file_system.cpp +++ b/src/duckdb/src/common/local_file_system.cpp @@ -34,6 +34,24 @@ extern "C" WINBASEAPI BOOL WINAPI GetPhysicallyInstalledSystemMemory(PULONGLONG) #undef FILE_CREATE // woo mingw #endif +// includes for giving a better error message on lock conflicts +#if defined(__linux__) || defined(__APPLE__) +#include +#endif + +#if defined(__linux__) +#include +// See e.g.: +// https://opensource.apple.com/source/CarbonHeaders/CarbonHeaders-18.1/TargetConditionals.h.auto.html +#elif defined(__APPLE__) +#include // NOLINT +#if not(defined(TARGET_OS_IPHONE) && TARGET_OS_IPHONE == 1) // NOLINT +#include // NOLINT +#endif // NOLINT +#elif defined(_WIN32) +#include +#endif + namespace duckdb { static void AssertValidFileFlags(uint8_t flags) { @@ -162,6 +180,88 @@ static FileType GetFileTypeInternal(int fd) { // LCOV_EXCL_START } } // LCOV_EXCL_STOP +#if __APPLE__ && !TARGET_OS_IPHONE + +static string AdditionalProcessInfo(FileSystem &fs, pid_t pid) { + if (pid == getpid()) { + return "Lock is already held in current process, likely another DuckDB instance"; + } + + string process_name, process_owner; + // try to find out more about the process holding the lock + struct proc_bsdshortinfo proc; + if (proc_pidinfo(pid, PROC_PIDT_SHORTBSDINFO, 0, &proc, PROC_PIDT_SHORTBSDINFO_SIZE) == + PROC_PIDT_SHORTBSDINFO_SIZE) { + process_name = proc.pbsi_comm; // only a short version however, let's take it in case proc_pidpath() below fails + // try to get actual name of conflicting process owner + auto pw = getpwuid(proc.pbsi_uid); + if (pw) { + process_owner = pw->pw_name; + } + } + // try to get a better process name (full path) + char full_exec_path[PROC_PIDPATHINFO_MAXSIZE]; + if (proc_pidpath(pid, full_exec_path, PROC_PIDPATHINFO_MAXSIZE) > 0) { + // somehow could not get the path, lets use some sensible fallback + process_name = full_exec_path; + } + return StringUtil::Format("Conflicting lock is held in %s%s", + !process_name.empty() ? StringUtil::Format("%s (PID %d)", process_name, pid) + : StringUtil::Format("PID %d", pid), + !process_owner.empty() ? StringUtil::Format(" by user %s", process_owner) : ""); +} + +#elif __linux__ + +static string AdditionalProcessInfo(FileSystem &fs, pid_t pid) { + if (pid == getpid()) { + return "Lock is already held in current process, likely another DuckDB instance"; + } + string process_name, process_owner; + + try { + auto cmdline_file = fs.OpenFile(StringUtil::Format("/proc/%d/cmdline", pid), FileFlags::FILE_FLAGS_READ); + auto cmdline = cmdline_file->ReadLine(); + process_name = basename(const_cast(cmdline.c_str())); + } catch (std::exception &) { + // ignore + } + + // we would like to provide a full path to the executable if possible but we might not have rights + { + char exe_target[PATH_MAX]; + memset(exe_target, '\0', PATH_MAX); + auto proc_exe_link = StringUtil::Format("/proc/%d/exe", pid); + auto readlink_n = readlink(proc_exe_link.c_str(), exe_target, PATH_MAX); + if (readlink_n > 0) { + process_name = exe_target; + } + } + + // try to find out who created that process + try { + auto loginuid_file = fs.OpenFile(StringUtil::Format("/proc/%d/loginuid", pid), FileFlags::FILE_FLAGS_READ); + auto uid = std::stoi(loginuid_file->ReadLine()); + auto pw = getpwuid(uid); + if (pw) { + process_owner = pw->pw_name; + } + } catch (std::exception &) { + // ignore + } + + return StringUtil::Format("Conflicting lock is held in %s%s", + !process_name.empty() ? StringUtil::Format("%s (PID %d)", process_name, pid) + : StringUtil::Format("PID %d", pid), + !process_owner.empty() ? StringUtil::Format(" by user %s", process_owner) : ""); +} + +#else +static string AdditionalProcessInfo(FileSystem &fs, pid_t pid) { + return ""; +} +#endif + unique_ptr LocalFileSystem::OpenFile(const string &path_p, uint8_t flags, FileLockType lock_type, FileCompressionType compression, FileOpener *opener) { auto path = FileSystem::ExpandPath(path_p, opener); @@ -199,7 +299,7 @@ unique_ptr LocalFileSystem::OpenFile(const string &path_p, uint8_t f } if (flags & FileFlags::FILE_FLAGS_DIRECT_IO) { #if defined(__sun) && defined(__SVR4) - throw Exception("DIRECT_IO not supported on Solaris"); + throw InvalidInputException("DIRECT_IO not supported on Solaris"); #endif #if defined(__DARWIN__) || defined(__APPLE__) || defined(__OpenBSD__) // OSX does not have O_DIRECT, instead we need to use fcntl afterwards to support direct IO @@ -234,7 +334,26 @@ unique_ptr LocalFileSystem::OpenFile(const string &path_p, uint8_t f fl.l_len = 0; rc = fcntl(fd, F_SETLK, &fl); if (rc == -1) { - throw IOException("Could not set lock on file \"%s\": %s", path, strerror(errno)); + string message; + // try to find out who is holding the lock using F_GETLK + rc = fcntl(fd, F_GETLK, &fl); + if (rc == -1) { // fnctl does not want to help us + message = strerror(errno); + } else { + message = AdditionalProcessInfo(*this, fl.l_pid); + } + + if (lock_type == FileLockType::WRITE_LOCK) { + // maybe we can get a read lock instead and tell this to the user. + fl.l_type = F_RDLCK; + rc = fcntl(fd, F_SETLK, &fl); + if (rc != -1) { // success! + message += ". However, you would be able to open this database in read-only mode, e.g. by " + "using the -readonly parameter in the CLI"; + } + } + message += ". See also https://duckdb.org/faq#how-does-duckdb-handle-concurrency"; + throw IOException("Could not set lock on file \"%s\": %s", path, message); } } } @@ -515,6 +634,68 @@ struct WindowsFileHandle : public FileHandle { }; }; +static string AdditionalLockInfo(const std::wstring path) { + // try to find out if another process is holding the lock + + // init of the somewhat obscure "Windows Restart Manager" + // see also https://devblogs.microsoft.com/oldnewthing/20120217-00/?p=8283 + + DWORD session, status, reason; + WCHAR session_key[CCH_RM_SESSION_KEY + 1] = {0}; + + status = RmStartSession(&session, 0, session_key); + if (status != ERROR_SUCCESS) { + return ""; + } + + PCWSTR path_ptr = path.c_str(); + status = RmRegisterResources(session, 1, &path_ptr, 0, NULL, 0, NULL); + if (status != ERROR_SUCCESS) { + return ""; + } + UINT process_info_size_needed, process_info_size; + + // we first call with nProcInfo = 0 to find out how much to allocate + process_info_size = 0; + status = RmGetList(session, &process_info_size_needed, &process_info_size, NULL, &reason); + if (status != ERROR_MORE_DATA || process_info_size_needed == 0) { + return ""; + } + + // allocate + auto process_info_buffer = duckdb::unique_ptr(new RM_PROCESS_INFO[process_info_size_needed]); + auto process_info = process_info_buffer.get(); + + // now call again to get actual data + process_info_size = process_info_size_needed; + status = RmGetList(session, &process_info_size_needed, &process_info_size, process_info, &reason); + if (status != ERROR_SUCCESS || process_info_size == 0) { + return ""; + } + + string conflict_string = "File is already open in "; + + for (UINT process_idx = 0; process_idx < process_info_size; process_idx++) { + string process_name = WindowsUtil::UnicodeToUTF8(process_info[process_idx].strAppName); + auto pid = process_info[process_idx].Process.dwProcessId; + + // find out full path if possible + HANDLE process = OpenProcess(PROCESS_QUERY_LIMITED_INFORMATION, FALSE, pid); + if (process) { + WCHAR full_path[MAX_PATH]; + DWORD full_path_size = MAX_PATH; + if (QueryFullProcessImageNameW(process, 0, full_path, &full_path_size) && full_path_size <= MAX_PATH) { + process_name = WindowsUtil::UnicodeToUTF8(full_path); + } + CloseHandle(process); + } + conflict_string += StringUtil::Format("\n%s (PID %d)", process_name, pid); + } + + RmEndSession(session); + return conflict_string; +} + unique_ptr LocalFileSystem::OpenFile(const string &path_p, uint8_t flags, FileLockType lock_type, FileCompressionType compression, FileOpener *opener) { auto path = FileSystem::ExpandPath(path_p, opener); @@ -556,6 +737,12 @@ unique_ptr LocalFileSystem::OpenFile(const string &path_p, uint8_t f flags_and_attributes, NULL); if (hFile == INVALID_HANDLE_VALUE) { auto error = LocalFileSystem::GetLastErrorAsString(); + + auto better_error = AdditionalLockInfo(unicode_path); + if (!better_error.empty()) { + throw IOException(better_error); + } + throw IOException("Cannot open file \"%s\": %s", path.c_str(), error); } auto handle = make_uniq(*this, path.c_str(), hFile); diff --git a/src/duckdb/src/common/multi_file_reader.cpp b/src/duckdb/src/common/multi_file_reader.cpp index 15b4d30cc..bbb59705e 100644 --- a/src/duckdb/src/common/multi_file_reader.cpp +++ b/src/duckdb/src/common/multi_file_reader.cpp @@ -377,8 +377,7 @@ void UnionByName::CombineUnionTypes(const vector &col_names, const vecto if (union_find != union_names_map.end()) { // given same name , union_col's type must compatible with col's type auto ¤t_type = union_col_types[union_find->second]; - LogicalType compatible_type; - compatible_type = LogicalType::MaxLogicalType(current_type, sql_types[col]); + auto compatible_type = LogicalType::ForceMaxLogicalType(current_type, sql_types[col]); union_col_types[union_find->second] = compatible_type; } else { union_names_map[col_names[col]] = union_col_names.size(); diff --git a/src/duckdb/src/common/operator/cast_operators.cpp b/src/duckdb/src/common/operator/cast_operators.cpp index 065261337..69ed385b0 100644 --- a/src/duckdb/src/common/operator/cast_operators.cpp +++ b/src/duckdb/src/common/operator/cast_operators.cpp @@ -24,6 +24,8 @@ #include "fast_float/fast_float.h" #include "fmt/format.h" #include "duckdb/common/types/bit.hpp" +#include "duckdb/common/operator/integer_cast_operator.hpp" +#include "duckdb/common/operator/double_cast_operator.hpp" #include #include @@ -64,6 +66,11 @@ bool TryCast::Operation(bool input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(bool input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(bool input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -127,6 +134,11 @@ bool TryCast::Operation(int8_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(int8_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(int8_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -190,6 +202,11 @@ bool TryCast::Operation(int16_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(int16_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(int16_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -253,6 +270,11 @@ bool TryCast::Operation(int32_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(int32_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(int32_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -316,6 +338,11 @@ bool TryCast::Operation(int64_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(int64_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(int64_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -379,6 +406,11 @@ bool TryCast::Operation(hugeint_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(hugeint_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(hugeint_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -409,6 +441,74 @@ bool TryCast::Operation(hugeint_t input, double &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +//===--------------------------------------------------------------------===// +// Cast uhugeint_t -> Numeric +//===--------------------------------------------------------------------===// +template <> +bool TryCast::Operation(uhugeint_t input, bool &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, int8_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, int16_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, int32_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, int64_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, hugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, uint8_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, uint16_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, uint32_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, uint64_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, float &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + +template <> +bool TryCast::Operation(uhugeint_t input, double &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + //===--------------------------------------------------------------------===// // Cast uint8_t -> Numeric //===--------------------------------------------------------------------===// @@ -442,6 +542,11 @@ bool TryCast::Operation(uint8_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(uint8_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(uint8_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -505,6 +610,11 @@ bool TryCast::Operation(uint16_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(uint16_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(uint16_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -568,6 +678,11 @@ bool TryCast::Operation(uint32_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(uint32_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(uint32_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -631,6 +746,11 @@ bool TryCast::Operation(uint64_t input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(uint64_t input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(uint64_t input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -694,6 +814,11 @@ bool TryCast::Operation(float input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(float input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(float input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -757,6 +882,11 @@ bool TryCast::Operation(double input, hugeint_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); } +template <> +bool TryCast::Operation(double input, uhugeint_t &result, bool strict) { + return NumericTryCast::Operation(input, result, strict); +} + template <> bool TryCast::Operation(double input, uint8_t &result, bool strict) { return NumericTryCast::Operation(input, result, strict); @@ -790,422 +920,6 @@ bool TryCast::Operation(double input, double &result, bool strict) { //===--------------------------------------------------------------------===// // Cast String -> Numeric //===--------------------------------------------------------------------===// - -template -struct IntegerCastData { - using ResultType = T; - using StoreType = T; - ResultType result; -}; - -struct IntegerCastOperation { - template - static bool HandleDigit(T &state, uint8_t digit) { - using store_t = typename T::StoreType; - if (NEGATIVE) { - if (DUCKDB_UNLIKELY(state.result < (NumericLimits::Minimum() + digit) / 10)) { - return false; - } - state.result = state.result * 10 - digit; - } else { - if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 10)) { - return false; - } - state.result = state.result * 10 + digit; - } - return true; - } - - template - static bool HandleHexDigit(T &state, uint8_t digit) { - using store_t = typename T::StoreType; - if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 16)) { - return false; - } - state.result = state.result * 16 + digit; - return true; - } - - template - static bool HandleBinaryDigit(T &state, uint8_t digit) { - using store_t = typename T::StoreType; - if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 2)) { - return false; - } - state.result = state.result * 2 + digit; - return true; - } - - template - static bool HandleExponent(T &state, int16_t exponent) { - // Simple integers don't deal with Exponents - return false; - } - - template - static bool HandleDecimal(T &state, uint8_t digit) { - // Simple integers don't deal with Decimals - return false; - } - - template - static bool Finalize(T &state) { - return true; - } -}; - -template -struct IntegerDecimalCastData { - using ResultType = T; - using StoreType = int64_t; - StoreType result; - StoreType decimal; - uint16_t decimal_digits; -}; - -template <> -struct IntegerDecimalCastData { - using ResultType = uint64_t; - using StoreType = uint64_t; - StoreType result; - StoreType decimal; - uint16_t decimal_digits; -}; - -struct IntegerDecimalCastOperation : IntegerCastOperation { - template - static bool HandleExponent(T &state, int16_t exponent) { - using store_t = typename T::StoreType; - - int16_t e = exponent; - // Negative Exponent - if (e < 0) { - while (state.result != 0 && e++ < 0) { - state.decimal = state.result % 10; - state.result /= 10; - } - if (state.decimal < 0) { - state.decimal = -state.decimal; - } - state.decimal_digits = 1; - return Finalize(state); - } - - // Positive Exponent - while (state.result != 0 && e-- > 0) { - if (!TryMultiplyOperator::Operation(state.result, (store_t)10, state.result)) { - return false; - } - } - - if (state.decimal == 0) { - return Finalize(state); - } - - // Handle decimals - e = exponent - state.decimal_digits; - store_t remainder = 0; - if (e < 0) { - if (static_cast(-e) <= NumericLimits::Digits()) { - store_t power = 1; - while (e++ < 0) { - power *= 10; - } - remainder = state.decimal % power; - state.decimal /= power; - } else { - state.decimal = 0; - } - } else { - while (e-- > 0) { - if (!TryMultiplyOperator::Operation(state.decimal, (store_t)10, state.decimal)) { - return false; - } - } - } - - state.decimal_digits -= exponent; - - if (NEGATIVE) { - if (!TrySubtractOperator::Operation(state.result, state.decimal, state.result)) { - return false; - } - } else if (!TryAddOperator::Operation(state.result, state.decimal, state.result)) { - return false; - } - state.decimal = remainder; - return Finalize(state); - } - - template - static bool HandleDecimal(T &state, uint8_t digit) { - using store_t = typename T::StoreType; - if (DUCKDB_UNLIKELY(state.decimal > (NumericLimits::Maximum() - digit) / 10)) { - // Simply ignore any more decimals - return true; - } - state.decimal_digits++; - state.decimal = state.decimal * 10 + digit; - return true; - } - - template - static bool Finalize(T &state) { - using result_t = typename T::ResultType; - using store_t = typename T::StoreType; - - result_t tmp; - if (!TryCast::Operation(state.result, tmp)) { - return false; - } - - while (state.decimal > 10) { - state.decimal /= 10; - state.decimal_digits--; - } - - bool success = true; - if (state.decimal_digits == 1 && state.decimal >= 5) { - if (NEGATIVE) { - success = TrySubtractOperator::Operation(tmp, (result_t)1, tmp); - } else { - success = TryAddOperator::Operation(tmp, (result_t)1, tmp); - } - } - state.result = tmp; - return success; - } -}; - -template -static bool IntegerCastLoop(const char *buf, idx_t len, T &result, bool strict) { - idx_t start_pos; - if (NEGATIVE) { - start_pos = 1; - } else { - if (*buf == '+') { - if (strict) { - // leading plus is not allowed in strict mode - return false; - } - start_pos = 1; - } else { - start_pos = 0; - } - } - idx_t pos = start_pos; - while (pos < len) { - if (!StringUtil::CharacterIsDigit(buf[pos])) { - // not a digit! - if (buf[pos] == decimal_separator) { - if (strict) { - return false; - } - bool number_before_period = pos > start_pos; - // decimal point: we accept decimal values for integers as well - // we just truncate them - // make sure everything after the period is a number - pos++; - idx_t start_digit = pos; - while (pos < len) { - if (!StringUtil::CharacterIsDigit(buf[pos])) { - break; - } - if (!OP::template HandleDecimal(result, buf[pos] - '0')) { - return false; - } - pos++; - } - // make sure there is either (1) one number after the period, or (2) one number before the period - // i.e. we accept "1." and ".1" as valid numbers, but not "." - if (!(number_before_period || pos > start_digit)) { - return false; - } - if (pos >= len) { - break; - } - } - if (StringUtil::CharacterIsSpace(buf[pos])) { - // skip any trailing spaces - while (++pos < len) { - if (!StringUtil::CharacterIsSpace(buf[pos])) { - return false; - } - } - break; - } - if (ALLOW_EXPONENT) { - if (buf[pos] == 'e' || buf[pos] == 'E') { - if (pos == start_pos) { - return false; - } - pos++; - if (pos >= len) { - return false; - } - using ExponentData = IntegerCastData; - ExponentData exponent {}; - int negative = buf[pos] == '-'; - if (negative) { - if (!IntegerCastLoop( - buf + pos, len - pos, exponent, strict)) { - return false; - } - } else { - if (!IntegerCastLoop( - buf + pos, len - pos, exponent, strict)) { - return false; - } - } - return OP::template HandleExponent(result, exponent.result); - } - } - return false; - } - uint8_t digit = buf[pos++] - '0'; - if (!OP::template HandleDigit(result, digit)) { - return false; - } - } - if (!OP::template Finalize(result)) { - return false; - } - return pos > start_pos; -} - -template -static bool IntegerHexCastLoop(const char *buf, idx_t len, T &result, bool strict) { - if (ALLOW_EXPONENT || NEGATIVE) { - return false; - } - idx_t start_pos = 1; - idx_t pos = start_pos; - char current_char; - while (pos < len) { - current_char = StringUtil::CharacterToLower(buf[pos]); - if (!StringUtil::CharacterIsHex(current_char)) { - return false; - } - uint8_t digit; - if (current_char >= 'a') { - digit = current_char - 'a' + 10; - } else { - digit = current_char - '0'; - } - pos++; - if (!OP::template HandleHexDigit(result, digit)) { - return false; - } - } - if (!OP::template Finalize(result)) { - return false; - } - return pos > start_pos; -} - -template -static bool IntegerBinaryCastLoop(const char *buf, idx_t len, T &result, bool strict) { - if (ALLOW_EXPONENT || NEGATIVE) { - return false; - } - idx_t start_pos = 1; - idx_t pos = start_pos; - uint8_t digit; - char current_char; - while (pos < len) { - current_char = buf[pos]; - if (current_char == '_' && pos > start_pos) { - // skip underscore, if it is not the first character - pos++; - if (pos == len) { - // we cant end on an underscore either - return false; - } - continue; - } else if (current_char == '0') { - digit = 0; - } else if (current_char == '1') { - digit = 1; - } else { - return false; - } - pos++; - if (!OP::template HandleBinaryDigit(result, digit)) { - return false; - } - } - if (!OP::template Finalize(result)) { - return false; - } - return pos > start_pos; -} - -template -static bool TryIntegerCast(const char *buf, idx_t len, T &result, bool strict) { - // skip any spaces at the start - while (len > 0 && StringUtil::CharacterIsSpace(*buf)) { - buf++; - len--; - } - if (len == 0) { - return false; - } - if (ZERO_INITIALIZE) { - memset(&result, 0, sizeof(T)); - } - // if the number is negative, we set the negative flag and skip the negative sign - if (*buf == '-') { - if (!IS_SIGNED) { - // Need to check if its not -0 - idx_t pos = 1; - while (pos < len) { - if (buf[pos++] != '0') { - return false; - } - } - } - return IntegerCastLoop(buf, len, result, strict); - } - if (len > 1 && *buf == '0') { - if (buf[1] == 'x' || buf[1] == 'X') { - // If it starts with 0x or 0X, we parse it as a hex value - buf++; - len--; - return IntegerHexCastLoop(buf, len, result, strict); - } else if (buf[1] == 'b' || buf[1] == 'B') { - // If it starts with 0b or 0B, we parse it as a binary value - buf++; - len--; - return IntegerBinaryCastLoop(buf, len, result, strict); - } else if (strict && StringUtil::CharacterIsDigit(buf[1])) { - // leading zeros are not allowed in strict mode - return false; - } - } - return IntegerCastLoop(buf, len, result, strict); -} - -template -static inline bool TrySimpleIntegerCast(const char *buf, idx_t len, T &result, bool strict) { - IntegerCastData simple_data; - if (TryIntegerCast, IS_SIGNED, false, IntegerCastOperation>(buf, len, simple_data, strict)) { - result = (T)simple_data.result; - return true; - } - - // Simple integer cast failed, try again with decimals/exponents included - // FIXME: This could definitely be improved as some extra work is being done here. It is more important that - // "normal" integers (without exponent/decimals) are still being parsed quickly. - IntegerDecimalCastData cast_data; - if (TryIntegerCast, IS_SIGNED, true, IntegerDecimalCastOperation>(buf, len, cast_data, - strict)) { - result = (T)cast_data.result; - return true; - } - return false; -} - template <> bool TryCast::Operation(string_t input, bool &result, bool strict) { auto input_data = input.GetData(); @@ -1284,44 +998,6 @@ bool TryCast::Operation(string_t input, uint64_t &result, bool strict) { return TrySimpleIntegerCast(input.GetData(), input.GetSize(), result, strict); } -template -static bool TryDoubleCast(const char *buf, idx_t len, T &result, bool strict) { - // skip any spaces at the start - while (len > 0 && StringUtil::CharacterIsSpace(*buf)) { - buf++; - len--; - } - if (len == 0) { - return false; - } - if (*buf == '+') { - if (strict) { - // plus is not allowed in strict mode - return false; - } - buf++; - len--; - } - if (strict && len >= 2) { - if (buf[0] == '0' && StringUtil::CharacterIsDigit(buf[1])) { - // leading zeros are not allowed in strict mode - return false; - } - } - auto endptr = buf + len; - auto parse_result = duckdb_fast_float::from_chars(buf, buf + len, result, decimal_separator); - if (parse_result.ec != std::errc()) { - return false; - } - auto current_end = parse_result.ptr; - if (!strict) { - while (current_end < endptr && StringUtil::CharacterIsSpace(*current_end)) { - current_end++; - } - } - return current_end == endptr; -} - template <> bool TryCast::Operation(string_t input, float &result, bool strict) { return TryDoubleCast(input.GetData(), input.GetSize(), result, strict); @@ -1334,7 +1010,7 @@ bool TryCast::Operation(string_t input, double &result, bool strict) { template <> bool TryCastErrorMessageCommaSeparated::Operation(string_t input, float &result, string *error_message, bool strict) { - if (!TryDoubleCast(input.GetData(), input.GetSize(), result, strict)) { + if (!TryDoubleCast(input.GetData(), input.GetSize(), result, strict, ',')) { HandleCastError::AssignError(StringUtil::Format("Could not cast string to float: \"%s\"", input.GetString()), error_message); return false; @@ -1344,7 +1020,7 @@ bool TryCastErrorMessageCommaSeparated::Operation(string_t input, float &result, template <> bool TryCastErrorMessageCommaSeparated::Operation(string_t input, double &result, string *error_message, bool strict) { - if (!TryDoubleCast(input.GetData(), input.GetSize(), result, strict)) { + if (!TryDoubleCast(input.GetData(), input.GetSize(), result, strict, ',')) { HandleCastError::AssignError(StringUtil::Format("Could not cast string to double: \"%s\"", input.GetString()), error_message); return false; @@ -1706,6 +1382,17 @@ bool CastFromBitToNumeric::Operation(string_t input, hugeint_t &result, bool str return (true); } +template <> +bool CastFromBitToNumeric::Operation(string_t input, uhugeint_t &result, bool strict) { + D_ASSERT(input.GetSize() > 1); + + if (input.GetSize() - 1 > sizeof(uhugeint_t)) { + throw ConversionException("Bitstring doesn't fit inside of %s", GetTypeId()); + } + Bit::BitToNumeric(input, result); + return (true); +} + //===--------------------------------------------------------------------===// // Cast From UUID //===--------------------------------------------------------------------===// @@ -1836,7 +1523,7 @@ bool TryCastErrorMessage::Operation(string_t input, interval_t &result, string * } //===--------------------------------------------------------------------===// -// Cast From Hugeint +// Cast to hugeint / uhugeint //===--------------------------------------------------------------------===// // parsing hugeint from string is done a bit differently for performance reasons // for other integer types we keep track of a single value @@ -1845,12 +1532,16 @@ bool TryCastErrorMessage::Operation(string_t input, interval_t &result, string * // for that reason, we parse numbers first into an int64 value // when that value is full, we perform a HUGEINT multiplication to flush it into the hugeint // this takes the number of HUGEINT multiplications down from [0-38] to [0-2] + +template struct HugeIntCastData { - hugeint_t result; + using ResultType = T; + using Operation = OP; + ResultType result; int64_t intermediate; uint8_t digits; - hugeint_t decimal; + ResultType decimal; uint16_t decimal_total_digits; int64_t decimal_intermediate; uint16_t decimal_intermediate_digits; @@ -1863,11 +1554,11 @@ struct HugeIntCastData { if (digits > 38) { return false; } - if (!Hugeint::TryMultiply(result, Hugeint::POWERS_OF_TEN[digits], result)) { + if (!OP::TryMultiply(result, OP::POWERS_OF_TEN[digits], result)) { return false; } } - if (!Hugeint::AddInPlace(result, hugeint_t(intermediate))) { + if (!OP::TryAddInPlace(result, ResultType(intermediate))) { return false; } digits = 0; @@ -1883,11 +1574,11 @@ struct HugeIntCastData { if (decimal_intermediate_digits > 38) { return false; } - if (!Hugeint::TryMultiply(decimal, Hugeint::POWERS_OF_TEN[decimal_intermediate_digits], decimal)) { + if (!OP::TryMultiply(decimal, OP::POWERS_OF_TEN[decimal_intermediate_digits], decimal)) { return false; } } - if (!Hugeint::AddInPlace(decimal, hugeint_t(decimal_intermediate))) { + if (!OP::TryAddInPlace(decimal, ResultType(decimal_intermediate))) { return false; } decimal_total_digits += decimal_intermediate_digits; @@ -1932,6 +1623,7 @@ struct HugeIntegerCastOperation { template static bool HandleExponent(T &state, int32_t exponent) { + using result_t = typename T::ResultType; if (!state.Flush()) { return false; } @@ -1943,9 +1635,12 @@ struct HugeIntegerCastOperation { } // Negative Exponent - hugeint_t remainder = 0; + result_t remainder = 0; if (e < 0) { - state.result = Hugeint::DivMod(state.result, Hugeint::POWERS_OF_TEN[-e], remainder); + state.result = T::Operation::DivMod(state.result, T::Operation::POWERS_OF_TEN[-e], remainder); + if (remainder < 0) { + remainder *= -1; + } state.decimal = remainder; state.decimal_total_digits = -e; state.decimal_intermediate = 0; @@ -1955,7 +1650,7 @@ struct HugeIntegerCastOperation { // Positive Exponent if (state.result != 0) { - if (e > 38 || !TryMultiplyOperator::Operation(state.result, Hugeint::POWERS_OF_TEN[e], state.result)) { + if (e > 38 || !TryMultiplyOperator::Operation(state.result, T::Operation::POWERS_OF_TEN[e], state.result)) { return false; } } @@ -1968,10 +1663,11 @@ struct HugeIntegerCastOperation { e = exponent - state.decimal_total_digits; if (e < 0) { - state.decimal = Hugeint::DivMod(state.decimal, Hugeint::POWERS_OF_TEN[-e], remainder); + state.decimal = T::Operation::DivMod(state.decimal, T::Operation::POWERS_OF_TEN[-e], remainder); state.decimal_total_digits -= (exponent); } else { - if (e > 38 || !TryMultiplyOperator::Operation(state.decimal, Hugeint::POWERS_OF_TEN[e], state.decimal)) { + if (e > 38 || + !TryMultiplyOperator::Operation(state.decimal, T::Operation::POWERS_OF_TEN[e], state.decimal)) { return false; } } @@ -2004,6 +1700,7 @@ struct HugeIntegerCastOperation { template static bool Finalize(T &state) { + using result_t = typename T::ResultType; if (!state.Flush() || !state.FlushDecimal()) { return false; } @@ -2014,17 +1711,17 @@ struct HugeIntegerCastOperation { // Get the first (left-most) digit of the decimals while (state.decimal_total_digits > 39) { - state.decimal /= Hugeint::POWERS_OF_TEN[39]; + state.decimal /= T::Operation::POWERS_OF_TEN[39]; state.decimal_total_digits -= 39; } D_ASSERT((state.decimal_total_digits - 1) >= 0 && (state.decimal_total_digits - 1) <= 39); - state.decimal /= Hugeint::POWERS_OF_TEN[state.decimal_total_digits - 1]; + state.decimal /= T::Operation::POWERS_OF_TEN[state.decimal_total_digits - 1]; - if (state.decimal >= 5 || state.decimal <= -5) { + if (state.decimal >= 5) { if (NEGATIVE) { - return TrySubtractOperator::Operation(state.result, hugeint_t(1), state.result); + return TrySubtractOperator::Operation(state.result, result_t(1), state.result); } else { - return TryAddOperator::Operation(state.result, hugeint_t(1), state.result); + return TryAddOperator::Operation(state.result, result_t(1), state.result); } } return true; @@ -2033,9 +1730,20 @@ struct HugeIntegerCastOperation { template <> bool TryCast::Operation(string_t input, hugeint_t &result, bool strict) { - HugeIntCastData state {}; - if (!TryIntegerCast(input.GetData(), input.GetSize(), state, - strict)) { + HugeIntCastData state {}; + if (!TryIntegerCast, true, true, HugeIntegerCastOperation>( + input.GetData(), input.GetSize(), state, strict)) { + return false; + } + result = state.result; + return true; +} + +template <> +bool TryCast::Operation(string_t input, uhugeint_t &result, bool strict) { + HugeIntCastData state {}; + if (!TryIntegerCast, false, true, HugeIntegerCastOperation>( + input.GetData(), input.GetSize(), state, strict)) { return false; } result = state.result; @@ -2628,6 +2336,46 @@ bool TryCastToDecimal::Operation(hugeint_t input, hugeint_t &result, string *err return HugeintToDecimalCast(input, result, error_message, width, scale); } +//===--------------------------------------------------------------------===// +// Uhugeint -> Decimal Cast +//===--------------------------------------------------------------------===// +template +bool UhugeintToDecimalCast(uhugeint_t input, DST &result, string *error_message, uint8_t width, uint8_t scale) { + // check for overflow + uhugeint_t max_width = Uhugeint::POWERS_OF_TEN[width - scale]; + if (input >= max_width) { + string error = StringUtil::Format("Could not cast value %s to DECIMAL(%d,%d)", input.ToString(), width, scale); + HandleCastError::AssignError(error, error_message); + return false; + } + result = Uhugeint::Cast(input * Uhugeint::POWERS_OF_TEN[scale]); + return true; +} + +template <> +bool TryCastToDecimal::Operation(uhugeint_t input, int16_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return UhugeintToDecimalCast(input, result, error_message, width, scale); +} + +template <> +bool TryCastToDecimal::Operation(uhugeint_t input, int32_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return UhugeintToDecimalCast(input, result, error_message, width, scale); +} + +template <> +bool TryCastToDecimal::Operation(uhugeint_t input, int64_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return UhugeintToDecimalCast(input, result, error_message, width, scale); +} + +template <> +bool TryCastToDecimal::Operation(uhugeint_t input, hugeint_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return UhugeintToDecimalCast(input, result, error_message, width, scale); +} + //===--------------------------------------------------------------------===// // Float/Double -> Decimal Cast //===--------------------------------------------------------------------===// @@ -2932,6 +2680,30 @@ bool TryCastFromDecimal::Operation(hugeint_t input, hugeint_t &result, string *e return TryCastHugeDecimalToNumeric(input, result, error_message, scale); } +//===--------------------------------------------------------------------===// +// Cast Decimal -> uhugeint_t +//===--------------------------------------------------------------------===// +template <> +bool TryCastFromDecimal::Operation(int16_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return TryCastDecimalToNumeric(input, result, error_message, scale); +} +template <> +bool TryCastFromDecimal::Operation(int32_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return TryCastDecimalToNumeric(input, result, error_message, scale); +} +template <> +bool TryCastFromDecimal::Operation(int64_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return TryCastDecimalToNumeric(input, result, error_message, scale); +} +template <> +bool TryCastFromDecimal::Operation(hugeint_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale) { + return TryCastHugeDecimalToNumeric(input, result, error_message, scale); +} + //===--------------------------------------------------------------------===// // Decimal -> Float/Double Cast //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/common/operator/convert_to_string.cpp b/src/duckdb/src/common/operator/convert_to_string.cpp index d2a20106e..758717759 100644 --- a/src/duckdb/src/common/operator/convert_to_string.cpp +++ b/src/duckdb/src/common/operator/convert_to_string.cpp @@ -51,6 +51,10 @@ string ConvertToString::Operation(hugeint_t input) { return StandardStringCast(input); } template <> +string ConvertToString::Operation(uhugeint_t input) { + return StandardStringCast(input); +} +template <> string ConvertToString::Operation(float input) { return StandardStringCast(input); } diff --git a/src/duckdb/src/common/operator/string_cast.cpp b/src/duckdb/src/common/operator/string_cast.cpp index 633c14085..2da6949ab 100644 --- a/src/duckdb/src/common/operator/string_cast.cpp +++ b/src/duckdb/src/common/operator/string_cast.cpp @@ -81,6 +81,11 @@ duckdb::string_t StringCast::Operation(hugeint_t input, Vector &vector) { return HugeintToStringCast::FormatSigned(input, vector); } +template <> +duckdb::string_t StringCast::Operation(uhugeint_t input, Vector &vector) { + return UhugeintToStringCast::Format(input, vector); +} + template <> duckdb::string_t StringCast::Operation(date_t input, Vector &vector) { if (input == date_t::infinity()) { diff --git a/src/duckdb/src/common/preserved_error.cpp b/src/duckdb/src/common/preserved_error.cpp deleted file mode 100644 index 7e7766017..000000000 --- a/src/duckdb/src/common/preserved_error.cpp +++ /dev/null @@ -1,87 +0,0 @@ -#include "duckdb/common/preserved_error.hpp" -#include "duckdb/common/exception.hpp" - -#include "duckdb/common/string_util.hpp" -#include "duckdb/common/to_string.hpp" -#include "duckdb/common/types.hpp" - -namespace duckdb { - -PreservedError::PreservedError() : initialized(false), exception_instance(nullptr) { -} - -PreservedError::PreservedError(const Exception &exception) - : initialized(true), type(exception.type), raw_message(SanitizeErrorMessage(exception.RawMessage())), - exception_instance(exception.Copy()) { -} - -PreservedError::PreservedError(const string &message) - : initialized(true), type(ExceptionType::INVALID), raw_message(SanitizeErrorMessage(message)), - exception_instance(nullptr) { - // Given a message in the form: xxxxx Error: yyyyy - // Try to match xxxxxxx with known error so to potentially reconstruct the original error type - auto position_semicolon = raw_message.find(':'); - if (position_semicolon == std::string::npos) { - // Semicolon not found, bail out - return; - } - if (position_semicolon + 2 >= raw_message.size()) { - // Not enough characters afterward, bail out - return; - } - string err = raw_message.substr(0, position_semicolon); - string msg = raw_message.substr(position_semicolon + 2); - if (err.size() > 6 && err.substr(err.size() - 6) == " Error" && !msg.empty()) { - ExceptionType new_type = Exception::StringToExceptionType(err.substr(0, err.size() - 6)); - if (new_type != type) { - type = new_type; - raw_message = msg; - } - } -} - -const string &PreservedError::Message() { - if (final_message.empty()) { - final_message = Exception::ExceptionTypeToString(type) + " Error: " + raw_message; - } - return final_message; -} - -string PreservedError::SanitizeErrorMessage(string error) { - return StringUtil::Replace(std::move(error), string("\0", 1), "\\0"); -} - -void PreservedError::Throw(const string &prepended_message) const { - D_ASSERT(initialized); - if (!prepended_message.empty()) { - string new_message = prepended_message + raw_message; - Exception::ThrowAsTypeWithMessage(type, new_message, exception_instance); - } - Exception::ThrowAsTypeWithMessage(type, raw_message, exception_instance); -} - -const ExceptionType &PreservedError::Type() const { - D_ASSERT(initialized); - return this->type; -} - -PreservedError &PreservedError::AddToMessage(const string &prepended_message) { - raw_message = prepended_message + raw_message; - return *this; -} - -PreservedError::operator bool() const { - return initialized; -} - -bool PreservedError::operator==(const PreservedError &other) const { - if (initialized != other.initialized) { - return false; - } - if (type != other.type) { - return false; - } - return raw_message == other.raw_message; -} - -} // namespace duckdb diff --git a/src/duckdb/src/common/progress_bar/terminal_progress_bar_display.cpp b/src/duckdb/src/common/progress_bar/terminal_progress_bar_display.cpp index a4b10a4d7..e20797125 100644 --- a/src/duckdb/src/common/progress_bar/terminal_progress_bar_display.cpp +++ b/src/duckdb/src/common/progress_bar/terminal_progress_bar_display.cpp @@ -4,13 +4,17 @@ namespace duckdb { -void TerminalProgressBarDisplay::PrintProgressInternal(int percentage) { +int32_t TerminalProgressBarDisplay::NormalizePercentage(double percentage) { if (percentage > 100) { - percentage = 100; + return 100; } if (percentage < 0) { - percentage = 0; + return 0; } + return int32_t(percentage); +} + +void TerminalProgressBarDisplay::PrintProgressInternal(int32_t percentage) { string result; // we divide the number of blocks by the percentage // 0% = 0 @@ -53,8 +57,13 @@ void TerminalProgressBarDisplay::PrintProgressInternal(int percentage) { } void TerminalProgressBarDisplay::Update(double percentage) { - PrintProgressInternal(percentage); + auto percentage_int = NormalizePercentage(percentage); + if (percentage_int == rendered_percentage) { + return; + } + PrintProgressInternal(percentage_int); Printer::Flush(OutputStream::STREAM_STDOUT); + rendered_percentage = percentage_int; } void TerminalProgressBarDisplay::Finish() { diff --git a/src/duckdb/src/common/row_operations/row_aggregate.cpp b/src/duckdb/src/common/row_operations/row_aggregate.cpp index 6c89d8870..f6e9e6cbb 100644 --- a/src/duckdb/src/common/row_operations/row_aggregate.cpp +++ b/src/duckdb/src/common/row_operations/row_aggregate.cpp @@ -82,7 +82,8 @@ void RowOperations::CombineStates(RowOperationsState &state, TupleDataLayout &la for (auto &aggr : layout.GetAggregates()) { D_ASSERT(aggr.function.combine); - AggregateInputData aggr_input_data(aggr.GetFunctionData(), state.allocator); + AggregateInputData aggr_input_data(aggr.GetFunctionData(), state.allocator, + AggregateCombineType::ALLOW_DESTRUCTIVE); aggr.function.combine(sources, targets, aggr_input_data, count); // Move to the next aggregate states diff --git a/src/duckdb/src/common/row_operations/row_gather.cpp b/src/duckdb/src/common/row_operations/row_gather.cpp index c04529024..ee2d1cb02 100644 --- a/src/duckdb/src/common/row_operations/row_gather.cpp +++ b/src/duckdb/src/common/row_operations/row_gather.cpp @@ -9,6 +9,7 @@ #include "duckdb/common/types/row/row_data_collection.hpp" #include "duckdb/common/types/row/row_layout.hpp" #include "duckdb/common/types/row/tuple_data_layout.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -135,6 +136,9 @@ void RowOperations::Gather(Vector &rows, const SelectionVector &row_sel, Vector case PhysicalType::UINT64: TemplatedGatherLoop(rows, row_sel, col, col_sel, count, layout, col_no, build_size); break; + case PhysicalType::UINT128: + TemplatedGatherLoop(rows, row_sel, col, col_sel, count, layout, col_no, build_size); + break; case PhysicalType::BOOL: case PhysicalType::INT8: TemplatedGatherLoop(rows, row_sel, col, col_sel, count, layout, col_no, build_size); 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 b709f054b..881178512 100644 --- a/src/duckdb/src/common/row_operations/row_heap_gather.cpp +++ b/src/duckdb/src/common/row_operations/row_heap_gather.cpp @@ -264,6 +264,9 @@ void RowOperations::HeapGather(Vector &v, const idx_t &vcount, const SelectionVe case PhysicalType::INT128: TemplatedHeapGather(v, vcount, sel, key_locations); break; + case PhysicalType::UINT128: + TemplatedHeapGather(v, vcount, sel, key_locations); + break; case PhysicalType::FLOAT: TemplatedHeapGather(v, vcount, sel, key_locations); break; 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 6665442f3..6b13bfffc 100644 --- a/src/duckdb/src/common/row_operations/row_heap_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_heap_scatter.cpp @@ -1,6 +1,7 @@ #include "duckdb/common/helper.hpp" #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -539,6 +540,9 @@ void RowOperations::HeapScatterVData(UnifiedVectorFormat &vdata, PhysicalType ty case PhysicalType::INT128: TemplatedHeapScatter(vdata, sel, ser_count, col_idx, key_locations, validitymask_locations, offset); break; + case PhysicalType::UINT128: + TemplatedHeapScatter(vdata, sel, ser_count, col_idx, key_locations, validitymask_locations, offset); + break; case PhysicalType::FLOAT: TemplatedHeapScatter(vdata, sel, ser_count, col_idx, key_locations, validitymask_locations, offset); break; diff --git a/src/duckdb/src/common/row_operations/row_matcher.cpp b/src/duckdb/src/common/row_operations/row_matcher.cpp index cca637209..a74e2ef5c 100644 --- a/src/duckdb/src/common/row_operations/row_matcher.cpp +++ b/src/duckdb/src/common/row_operations/row_matcher.cpp @@ -238,6 +238,8 @@ MatchFunction RowMatcher::GetMatchFunction(const LogicalType &type, const Expres return GetMatchFunction(predicate); case PhysicalType::UINT64: return GetMatchFunction(predicate); + case PhysicalType::UINT128: + return GetMatchFunction(predicate); case PhysicalType::FLOAT: return GetMatchFunction(predicate); case PhysicalType::DOUBLE: 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 fc4ef8f54..01557ca45 100644 --- a/src/duckdb/src/common/row_operations/row_radix_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_radix_scatter.cpp @@ -2,6 +2,7 @@ #include "duckdb/common/radix.hpp" #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -286,6 +287,9 @@ void RowOperations::RadixScatter(Vector &v, idx_t vcount, const SelectionVector case PhysicalType::INT128: TemplatedRadixScatter(vdata, sel, ser_count, key_locations, desc, has_null, nulls_first, offset); break; + case PhysicalType::UINT128: + TemplatedRadixScatter(vdata, sel, ser_count, key_locations, desc, has_null, nulls_first, offset); + break; case PhysicalType::FLOAT: TemplatedRadixScatter(vdata, sel, ser_count, key_locations, desc, has_null, nulls_first, offset); break; diff --git a/src/duckdb/src/common/row_operations/row_scatter.cpp b/src/duckdb/src/common/row_operations/row_scatter.cpp index 14055e47a..6d11c4375 100644 --- a/src/duckdb/src/common/row_operations/row_scatter.cpp +++ b/src/duckdb/src/common/row_operations/row_scatter.cpp @@ -12,6 +12,7 @@ #include "duckdb/common/types/row/row_layout.hpp" #include "duckdb/common/types/selection_vector.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -204,6 +205,9 @@ void RowOperations::Scatter(DataChunk &columns, UnifiedVectorFormat col_data[], case PhysicalType::INT128: TemplatedScatter(col, rows, sel, count, col_offset, col_no); break; + case PhysicalType::UINT128: + TemplatedScatter(col, rows, sel, count, col_offset, col_no); + break; case PhysicalType::FLOAT: TemplatedScatter(col, rows, sel, count, col_offset, col_no); break; diff --git a/src/duckdb/src/common/serializer/binary_deserializer.cpp b/src/duckdb/src/common/serializer/binary_deserializer.cpp index 6b31146c8..a13e35781 100644 --- a/src/duckdb/src/common/serializer/binary_deserializer.cpp +++ b/src/duckdb/src/common/serializer/binary_deserializer.cpp @@ -124,6 +124,12 @@ hugeint_t BinaryDeserializer::ReadHugeInt() { return hugeint_t(upper, lower); } +uhugeint_t BinaryDeserializer::ReadUhugeInt() { + auto upper = VarIntDecode(); + auto lower = VarIntDecode(); + return uhugeint_t(upper, lower); +} + void BinaryDeserializer::ReadDataPtr(data_ptr_t &ptr_p, idx_t count) { auto len = VarIntDecode(); if (len != count) { diff --git a/src/duckdb/src/common/serializer/binary_serializer.cpp b/src/duckdb/src/common/serializer/binary_serializer.cpp index 67f31b562..cf2b26969 100644 --- a/src/duckdb/src/common/serializer/binary_serializer.cpp +++ b/src/duckdb/src/common/serializer/binary_serializer.cpp @@ -136,6 +136,11 @@ void BinarySerializer::WriteValue(hugeint_t value) { VarIntEncode(value.lower); } +void BinarySerializer::WriteValue(uhugeint_t value) { + VarIntEncode(value.upper); + VarIntEncode(value.lower); +} + void BinarySerializer::WriteValue(float value) { Write(value); } diff --git a/src/duckdb/src/common/serializer/serializer.cpp b/src/duckdb/src/common/serializer/serializer.cpp index efeaa9660..e74461203 100644 --- a/src/duckdb/src/common/serializer/serializer.cpp +++ b/src/duckdb/src/common/serializer/serializer.cpp @@ -1,4 +1,5 @@ #include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/types/value.hpp" namespace duckdb { @@ -12,6 +13,20 @@ void Serializer::WriteValue(const vector &vec) { OnListEnd(); } +template <> +void Serializer::WritePropertyWithDefault(const field_id_t field_id, const char *tag, const Value &value, + const Value &&default_value) { + // If current value is default, don't write it + if (!serialize_default_values && ValueOperations::NotDistinctFrom(value, default_value)) { + OnOptionalPropertyBegin(field_id, tag, false); + OnOptionalPropertyEnd(false); + return; + } + OnOptionalPropertyBegin(field_id, tag, true); + WriteValue(value); + OnOptionalPropertyEnd(true); +} + void Serializer::List::WriteElement(data_ptr_t ptr, idx_t size) { serializer.WriteDataPtr(ptr, size); } diff --git a/src/duckdb/src/common/sort/comparators.cpp b/src/duckdb/src/common/sort/comparators.cpp index 0533fe2d1..7084a3f99 100644 --- a/src/duckdb/src/common/sort/comparators.cpp +++ b/src/duckdb/src/common/sort/comparators.cpp @@ -2,6 +2,7 @@ #include "duckdb/common/fast_mem.hpp" #include "duckdb/common/sort/sort.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -135,6 +136,8 @@ int Comparators::CompareValAndAdvance(data_ptr_t &l_ptr, data_ptr_t &r_ptr, cons return TemplatedCompareAndAdvance(l_ptr, r_ptr); case PhysicalType::INT128: return TemplatedCompareAndAdvance(l_ptr, r_ptr); + case PhysicalType::UINT128: + return TemplatedCompareAndAdvance(l_ptr, r_ptr); case PhysicalType::FLOAT: return TemplatedCompareAndAdvance(l_ptr, r_ptr); case PhysicalType::DOUBLE: @@ -387,6 +390,9 @@ int Comparators::CompareListAndAdvance(data_ptr_t &left_ptr, data_ptr_t &right_p case PhysicalType::INT128: comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, count); break; + case PhysicalType::UINT128: + comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, count); + break; case PhysicalType::FLOAT: comp_res = TemplatedCompareListLoop(left_ptr, right_ptr, left_validity, right_validity, count); break; diff --git a/src/duckdb/src/common/sort/sort_state.cpp b/src/duckdb/src/common/sort/sort_state.cpp index b9316c7a7..a91736367 100644 --- a/src/duckdb/src/common/sort/sort_state.cpp +++ b/src/duckdb/src/common/sort/sort_state.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/sort/sort.hpp" #include "duckdb/common/sort/sorted_block.hpp" +#include "duckdb/storage/buffer/buffer_pool.hpp" #include #include @@ -400,7 +401,7 @@ void GlobalSortState::PrepareMergePhase() { idx_t total_heap_size = std::accumulate(sorted_blocks.begin(), sorted_blocks.end(), (idx_t)0, [](idx_t a, const unique_ptr &b) { return a + b->HeapSize(); }); - if (external || (pinned_blocks.empty() && total_heap_size > 0.25 * buffer_manager.GetMaxMemory())) { + if (external || (pinned_blocks.empty() && total_heap_size > 0.25 * buffer_manager.GetQueryMaxMemory())) { external = true; } // Use the data that we have to determine which partition size to use during the merge diff --git a/src/duckdb/src/common/string_util.cpp b/src/duckdb/src/common/string_util.cpp index 6f3538c21..79d99a88e 100644 --- a/src/duckdb/src/common/string_util.cpp +++ b/src/duckdb/src/common/string_util.cpp @@ -405,4 +405,182 @@ string StringUtil::CandidatesErrorMessage(const vector &strings, const s return StringUtil::CandidatesMessage(closest_strings, message_prefix); } +static void SkipSpaces(const string &message, idx_t &pos) { + for (; pos < message.size() && StringUtil::CharacterIsSpace(message[pos]); pos++) { + } +} + +static bool MatchCharacter(const string &message, idx_t &pos, char c) { + if (pos >= message.size()) { + return false; + } + return message[pos] == c; +} + +static string ParseJSONValue(const string &message, idx_t &pos) { + string result; + if (!MatchCharacter(message, pos, '"')) { + // values need to start with a quote + D_ASSERT(0); + return result; + } + pos++; + for (; pos < message.size(); pos++) { + if (message[pos] == '\\') { + // escape + pos++; + if (pos >= message.size()) { + // escape at end of string!? + D_ASSERT(0); + return result; + } + switch (message[pos]) { + case 'r': + result += '\r'; + break; + case 'n': + result += '\n'; + break; + case 't': + result += '\t'; + break; + case 'b': + result += '\b'; + break; + case 'f': + result += '\f'; + break; + case '0': + result += '\0'; + break; + case '\\': + case '"': + case '/': + result += message[pos]; + break; + default: + // unsupported escape character + // NOTE: we do not support unicode escape sequences here + D_ASSERT(0); + result += message[pos]; + break; + } + } else if (message[pos] == '"') { + // end of message + pos++; + return result; + } else { + result += message[pos]; + } + } + // no end-of-value found + D_ASSERT(0); + return result; +} + +unordered_map StringUtil::ParseJSONMap(const string &json) { + unordered_map result; + if (json.empty()) { + return result; + } + idx_t pos = 0; + SkipSpaces(json, pos); + if (!MatchCharacter(json, pos, '{')) { + D_ASSERT(0); + return result; + } + pos++; + while (true) { + SkipSpaces(json, pos); + if (MatchCharacter(json, pos, '}')) { + // end of object + break; + } + if (!result.empty()) { + // objects are comma separated + if (!MatchCharacter(json, pos, ',')) { + D_ASSERT(0); + return result; + } + pos++; + } + string key = ParseJSONValue(json, pos); + SkipSpaces(json, pos); + if (!MatchCharacter(json, pos, ':')) { + D_ASSERT(0); + return result; + } + pos++; + string value = ParseJSONValue(json, pos); + auto entry = result.find(key); + if (entry != result.end()) { + // entry already exists + D_ASSERT(0); + continue; + } + result.insert(make_pair(std::move(key), std::move(value))); + } + return result; +} + +static void WriteJSONValue(const string &value, string &result) { + result += '"'; + for (auto c : value) { + // check for characters we need to escape + switch (c) { + case '\0': + result += "\\0"; + break; + case '\\': + result += "\\\\"; + break; + case '\b': + result += "\\b"; + break; + case '\f': + result += "\\f"; + break; + case '\t': + result += "\\t"; + break; + case '\r': + result += "\\r"; + break; + case '\n': + result += "\\n"; + break; + case '"': + result += "\\\""; + break; + default: + result += c; + break; + } + } + result += '"'; +} + +static void WriteJSONPair(const string &key, const string &value, string &result) { + WriteJSONValue(key, result); + result += ":"; + WriteJSONValue(value, result); +} + +string StringUtil::ToJSONMap(ExceptionType type, const string &message, const unordered_map &map) { + D_ASSERT(map.find("exception_type") == map.end()); + D_ASSERT(map.find("exception_message") == map.end()); + string result; + result += "{"; + // we always write exception type/message + WriteJSONPair("exception_type", Exception::ExceptionTypeToString(type), result); + result += ","; + WriteJSONPair("exception_message", message, result); + for (auto &entry : map) { + result += ","; + WriteJSONPair(entry.first, entry.second, result); + } + result += "}"; + return result; +} + } // namespace duckdb diff --git a/src/duckdb/src/common/tree_renderer.cpp b/src/duckdb/src/common/tree_renderer.cpp index 1ce04df6d..c3cd19803 100644 --- a/src/duckdb/src/common/tree_renderer.cpp +++ b/src/duckdb/src/common/tree_renderer.cpp @@ -1,12 +1,13 @@ #include "duckdb/common/tree_renderer.hpp" -#include "duckdb/planner/logical_operator.hpp" -#include "duckdb/execution/physical_operator.hpp" -#include "duckdb/common/string_util.hpp" + #include "duckdb/common/pair.hpp" -#include "duckdb/execution/operator/join/physical_delim_join.hpp" +#include "duckdb/common/string_util.hpp" #include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" +#include "duckdb/execution/operator/join/physical_delim_join.hpp" #include "duckdb/execution/operator/scan/physical_positional_scan.hpp" +#include "duckdb/execution/physical_operator.hpp" #include "duckdb/parallel/pipeline.hpp" +#include "duckdb/planner/logical_operator.hpp" #include "utf8proc_wrapper.hpp" #include @@ -378,7 +379,8 @@ class TreeChildrenIterator { template <> bool TreeChildrenIterator::HasChildren(const PhysicalOperator &op) { switch (op.type) { - case PhysicalOperatorType::DELIM_JOIN: + case PhysicalOperatorType::LEFT_DELIM_JOIN: + case PhysicalOperatorType::RIGHT_DELIM_JOIN: case PhysicalOperatorType::POSITIONAL_SCAN: return true; default: @@ -391,7 +393,7 @@ void TreeChildrenIterator::Iterate(const PhysicalOperator &op, for (auto &child : op.children) { callback(*child); } - if (op.type == PhysicalOperatorType::DELIM_JOIN) { + if (op.type == PhysicalOperatorType::LEFT_DELIM_JOIN || op.type == PhysicalOperatorType::RIGHT_DELIM_JOIN) { auto &delim = op.Cast(); callback(*delim.join); } else if ((op.type == PhysicalOperatorType::POSITIONAL_SCAN)) { diff --git a/src/duckdb/src/common/types.cpp b/src/duckdb/src/common/types.cpp index 8384d1084..8fbec1d57 100644 --- a/src/duckdb/src/common/types.cpp +++ b/src/duckdb/src/common/types.cpp @@ -1,17 +1,23 @@ #include "duckdb/common/types.hpp" #include "duckdb/catalog/catalog.hpp" +#include "duckdb/catalog/catalog_entry/type_catalog_entry.hpp" #include "duckdb/catalog/catalog_search_path.hpp" #include "duckdb/catalog/default/default_types.hpp" +#include "duckdb/common/enum_util.hpp" #include "duckdb/common/exception.hpp" +#include "duckdb/common/extra_type_info.hpp" #include "duckdb/common/limits.hpp" #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/decimal.hpp" #include "duckdb/common/types/hash.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/types/value.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/function/cast_rules.hpp" #include "duckdb/main/attached_database.hpp" @@ -21,11 +27,8 @@ #include "duckdb/main/database_manager.hpp" #include "duckdb/parser/keyword_helper.hpp" #include "duckdb/parser/parser.hpp" -#include "duckdb/common/extra_type_info.hpp" -#include "duckdb/common/serializer/deserializer.hpp" -#include "duckdb/common/enum_util.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/catalog/catalog_entry/type_catalog_entry.hpp" +#include "duckdb/main/config.hpp" + #include namespace duckdb { @@ -82,6 +85,8 @@ PhysicalType LogicalType::GetInternalType() { return PhysicalType::INT64; case LogicalTypeId::UBIGINT: return PhysicalType::UINT64; + case LogicalTypeId::UHUGEINT: + return PhysicalType::UINT128; case LogicalTypeId::HUGEINT: case LogicalTypeId::UUID: return PhysicalType::INT128; @@ -145,6 +150,8 @@ PhysicalType LogicalType::GetInternalType() { case LogicalTypeId::ANY: case LogicalTypeId::INVALID: case LogicalTypeId::UNKNOWN: + case LogicalTypeId::STRING_LITERAL: + case LogicalTypeId::INTEGER_LITERAL: return PhysicalType::INVALID; case LogicalTypeId::USER: return PhysicalType::UNKNOWN; @@ -172,6 +179,7 @@ constexpr const LogicalTypeId LogicalType::UINTEGER; constexpr const LogicalTypeId LogicalType::BIGINT; constexpr const LogicalTypeId LogicalType::UBIGINT; constexpr const LogicalTypeId LogicalType::HUGEINT; +constexpr const LogicalTypeId LogicalType::UHUGEINT; constexpr const LogicalTypeId LogicalType::UUID; constexpr const LogicalTypeId LogicalType::FLOAT; constexpr const LogicalTypeId LogicalType::DOUBLE; @@ -207,14 +215,16 @@ const vector LogicalType::Numeric() { vector types = {LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::HUGEINT, LogicalType::FLOAT, LogicalType::DOUBLE, LogicalTypeId::DECIMAL, LogicalType::UTINYINT, - LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT}; + LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT, + LogicalType::UHUGEINT}; return types; } const vector LogicalType::Integral() { vector types = {LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::HUGEINT, LogicalType::UTINYINT, - LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT}; + LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT, + LogicalType::UHUGEINT}; return types; } @@ -225,13 +235,14 @@ const vector LogicalType::Real() { const vector LogicalType::AllTypes() { vector types = { - LogicalType::BOOLEAN, LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, - LogicalType::BIGINT, LogicalType::DATE, LogicalType::TIMESTAMP, LogicalType::DOUBLE, - LogicalType::FLOAT, LogicalType::VARCHAR, LogicalType::BLOB, LogicalType::BIT, - 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::ARRAY}; + LogicalType::BOOLEAN, LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, + LogicalType::BIGINT, LogicalType::DATE, LogicalType::TIMESTAMP, LogicalType::DOUBLE, + LogicalType::FLOAT, LogicalType::VARCHAR, LogicalType::BLOB, LogicalType::BIT, + LogicalType::INTERVAL, LogicalType::HUGEINT, LogicalTypeId::DECIMAL, LogicalType::UTINYINT, + LogicalType::USMALLINT, LogicalType::UINTEGER, LogicalType::UBIGINT, LogicalType::UHUGEINT, + LogicalType::TIME, LogicalTypeId::LIST, LogicalTypeId::STRUCT, LogicalType::TIME_TZ, + LogicalType::TIMESTAMP_TZ, LogicalTypeId::MAP, LogicalTypeId::UNION, LogicalType::UUID, + LogicalTypeId::ARRAY}; return types; } @@ -260,6 +271,8 @@ string TypeIdToString(PhysicalType type) { return "UINT64"; case PhysicalType::INT128: return "INT128"; + case PhysicalType::UINT128: + return "UINT128"; case PhysicalType::FLOAT: return "FLOAT"; case PhysicalType::DOUBLE: @@ -308,6 +321,8 @@ idx_t GetTypeIdSize(PhysicalType type) { return sizeof(uint64_t); case PhysicalType::INT128: return sizeof(hugeint_t); + case PhysicalType::UINT128: + return sizeof(uhugeint_t); case PhysicalType::FLOAT: return sizeof(float); case PhysicalType::DOUBLE: @@ -329,16 +344,19 @@ idx_t GetTypeIdSize(PhysicalType type) { bool TypeIsConstantSize(PhysicalType type) { return (type >= PhysicalType::BOOL && type <= PhysicalType::DOUBLE) || type == PhysicalType::INTERVAL || - type == PhysicalType::INT128; + type == PhysicalType::INT128 || type == PhysicalType::UINT128; } bool TypeIsIntegral(PhysicalType type) { - return (type >= PhysicalType::UINT8 && type <= PhysicalType::INT64) || type == PhysicalType::INT128; + return (type >= PhysicalType::UINT8 && type <= PhysicalType::INT64) || type == PhysicalType::INT128 || + type == PhysicalType::UINT128; } bool TypeIsNumeric(PhysicalType type) { - return (type >= PhysicalType::UINT8 && type <= PhysicalType::DOUBLE) || type == PhysicalType::INT128; + return (type >= PhysicalType::UINT8 && type <= PhysicalType::DOUBLE) || type == PhysicalType::INT128 || + type == PhysicalType::UINT128; } bool TypeIsInteger(PhysicalType type) { - return (type >= PhysicalType::UINT8 && type <= PhysicalType::INT64) || type == PhysicalType::INT128; + return (type >= PhysicalType::UINT8 && type <= PhysicalType::INT64) || type == PhysicalType::INT128 || + type == PhysicalType::UINT128; } string LogicalType::ToString() const { @@ -353,10 +371,15 @@ string LogicalType::ToString() const { if (!type_info_) { return "STRUCT"; } + auto is_unnamed = StructType::IsUnnamed(*this); auto &child_types = StructType::GetChildTypes(*this); string ret = "STRUCT("; for (size_t i = 0; i < child_types.size(); i++) { - ret += StringUtil::Format("%s %s", SQLIdentifier(child_types[i].first), child_types[i].second); + if (is_unnamed) { + ret += child_types[i].second.ToString(); + } else { + ret += StringUtil::Format("%s %s", SQLIdentifier(child_types[i].first), child_types[i].second); + } if (i < child_types.size() - 1) { ret += ", "; } @@ -514,6 +537,7 @@ bool LogicalType::IsIntegral() const { case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: case LogicalTypeId::HUGEINT: + case LogicalTypeId::UHUGEINT: return true; default: return false; @@ -534,6 +558,7 @@ bool LogicalType::IsNumeric() const { case LogicalTypeId::USMALLINT: case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: + case LogicalTypeId::UHUGEINT: return true; default: return false; @@ -600,10 +625,18 @@ bool LogicalType::GetDecimalProperties(uint8_t &width, uint8_t &scale) const { width = 38; scale = 0; break; + case LogicalTypeId::UHUGEINT: + // hugeint: max size decimal (38, 0) + // note that a uhugeint is not guaranteed to fit in this + width = 38; + scale = 0; + break; case LogicalTypeId::DECIMAL: width = DecimalType::GetWidth(*this); scale = DecimalType::GetScale(*this); break; + case LogicalTypeId::INTEGER_LITERAL: + return IntegerLiteral::GetType(*this).GetDecimalProperties(width, scale); default: // Nonsense values to ensure initialization width = 255u; @@ -652,22 +685,6 @@ static LogicalType CombineNumericTypes(const LogicalType &left, const LogicalTyp // arrange it so the left type is smaller to limit the number of options we need to check return CombineNumericTypes(right, left); } - if (CastRules::ImplicitCast(left, right) >= 0) { - // we can implicitly cast left to right, return right - //! Depending on the type, we might need to grow the `width` of the DECIMAL type - if (right.id() == LogicalTypeId::DECIMAL) { - return DecimalSizeCheck(left, right); - } - return right; - } - if (CastRules::ImplicitCast(right, left) >= 0) { - // we can implicitly cast right to left, return left - //! Depending on the type, we might need to grow the `width` of the DECIMAL type - if (left.id() == LogicalTypeId::DECIMAL) { - return DecimalSizeCheck(right, left); - } - return left; - } // we can't cast implicitly either way and types are not equal // this happens when left is signed and right is unsigned // e.g. INTEGER and UINTEGER @@ -685,47 +702,118 @@ static LogicalType CombineNumericTypes(const LogicalType &left, const LogicalTyp if (left.id() == LogicalTypeId::TINYINT || right.id() == LogicalTypeId::UTINYINT) { return LogicalType::SMALLINT; } - throw InternalException("Cannot combine these numeric types!?"); + + // No type is larger than (u)hugeint, so casting to double is required + // UHUGEINT is on the left because the enum is lower + if (left.id() == LogicalTypeId::UHUGEINT || right.id() == LogicalTypeId::HUGEINT) { + return LogicalType::DOUBLE; + } + throw InternalException("Cannot combine these numeric types (%s & %s)", left.ToString(), right.ToString()); } -LogicalType LogicalType::MaxLogicalType(const LogicalType &left, const LogicalType &right) { - // we always prefer aliased types - if (!left.GetAlias().empty()) { - return left; +LogicalType LogicalType::NormalizeType(const LogicalType &type) { + switch (type.id()) { + case LogicalTypeId::STRING_LITERAL: + return LogicalType::VARCHAR; + case LogicalTypeId::INTEGER_LITERAL: + return IntegerLiteral::GetType(type); + default: + return type; + } +} + +template +static bool CombineUnequalTypes(const LogicalType &left, const LogicalType &right, LogicalType &result) { + // left and right are not equal + // for enums, match the varchar rules + if (left.id() == LogicalTypeId::ENUM) { + return OP::Operation(LogicalType::VARCHAR, right, result); + } else if (right.id() == LogicalTypeId::ENUM) { + return OP::Operation(left, LogicalType::VARCHAR, result); + } + // NULL/string literals/unknown (parameter) types always take the other type + LogicalTypeId other_types[] = {LogicalTypeId::UNKNOWN, LogicalTypeId::SQLNULL, LogicalTypeId::STRING_LITERAL}; + for (auto &other_type : other_types) { + if (left.id() == other_type) { + result = LogicalType::NormalizeType(right); + return true; + } else if (right.id() == other_type) { + result = LogicalType::NormalizeType(left); + return true; + } } - if (!right.GetAlias().empty()) { - return right; + + // for other types - use implicit cast rules to check if we can combine the types + auto left_to_right_cost = CastRules::ImplicitCast(left, right); + auto right_to_left_cost = CastRules::ImplicitCast(right, left); + if (left_to_right_cost >= 0 && (left_to_right_cost < right_to_left_cost || right_to_left_cost < 0)) { + // we can implicitly cast left to right, return right + //! Depending on the type, we might need to grow the `width` of the DECIMAL type + if (right.id() == LogicalTypeId::DECIMAL) { + result = DecimalSizeCheck(left, right); + } else { + result = right; + } + return true; } - if (left.id() != right.id() && left.IsNumeric() && right.IsNumeric()) { - return CombineNumericTypes(left, right); - } else if (left.id() == LogicalTypeId::UNKNOWN) { - return right; - } else if (right.id() == LogicalTypeId::UNKNOWN) { - return left; - } else if ((right.id() == LogicalTypeId::ENUM || left.id() == LogicalTypeId::ENUM) && right.id() != left.id()) { - // if one is an enum and the other is not, compare strings, not enums - // see https://github.com/duckdb/duckdb/issues/8561 - return LogicalTypeId::VARCHAR; - } else if (left.id() < right.id()) { - return right; + if (right_to_left_cost >= 0) { + // we can implicitly cast right to left, return left + //! Depending on the type, we might need to grow the `width` of the DECIMAL type + if (left.id() == LogicalTypeId::DECIMAL) { + result = DecimalSizeCheck(right, left); + } else { + result = left; + } + return true; } - if (right.id() < left.id()) { - return left; + // for integer literals - rerun the operation with the underlying type + if (left.id() == LogicalTypeId::INTEGER_LITERAL) { + return OP::Operation(IntegerLiteral::GetType(left), right, result); + } + if (right.id() == LogicalTypeId::INTEGER_LITERAL) { + return OP::Operation(left, IntegerLiteral::GetType(right), result); + } + // for unsigned/signed comparisons we have a few fallbacks + if (left.IsNumeric() && right.IsNumeric()) { + result = CombineNumericTypes(left, right); + return true; + } + if (left.id() == LogicalTypeId::BOOLEAN && right.IsIntegral()) { + result = right; + return true; + } + if (right.id() == LogicalTypeId::BOOLEAN && left.IsIntegral()) { + result = left; + return true; } + return false; +} + +template +static bool CombineEqualTypes(const LogicalType &left, const LogicalType &right, LogicalType &result) { // Since both left and right are equal we get the left type as our type_id for checks auto type_id = left.id(); - if (type_id == LogicalTypeId::ENUM) { + switch (type_id) { + case LogicalTypeId::STRING_LITERAL: + // two string literals convert to varchar + result = LogicalType::VARCHAR; + return true; + case LogicalTypeId::INTEGER_LITERAL: + // for two integer literals we unify the underlying types + return OP::Operation(IntegerLiteral::GetType(left), IntegerLiteral::GetType(right), result); + case LogicalTypeId::ENUM: // If both types are different ENUMs we do a string comparison. - return left == right ? left : LogicalType::VARCHAR; - } - if (type_id == LogicalTypeId::VARCHAR) { + result = left == right ? left : LogicalType::VARCHAR; + return true; + case LogicalTypeId::VARCHAR: // varchar: use type that has collation (if any) if (StringType::GetCollation(right).empty()) { - return left; + result = left; + } else { + result = right; } - return right; - } - if (type_id == LogicalTypeId::DECIMAL) { + return true; + case LogicalTypeId::DECIMAL: { // unify the width/scale so that the resulting decimal always fits // "width - scale" gives us the number of digits on the left side of the decimal point // "scale" gives us the number of digits allowed on the right of the decimal point @@ -740,52 +828,227 @@ LogicalType LogicalType::MaxLogicalType(const LogicalType &left, const LogicalTy width = DecimalType::MaxWidth(); scale = width - extra_width; } - return LogicalType::DECIMAL(width, scale); + result = LogicalType::DECIMAL(width, scale); + return true; } - if (type_id == LogicalTypeId::LIST) { + case LogicalTypeId::LIST: { // list: perform max recursively on child type - auto new_child = MaxLogicalType(ListType::GetChildType(left), ListType::GetChildType(right)); - return LogicalType::LIST(new_child); + LogicalType new_child; + if (!OP::Operation(ListType::GetChildType(left), ListType::GetChildType(right), new_child)) { + return false; + } + result = LogicalType::LIST(new_child); + return true; } - if (type_id == LogicalTypeId::ARRAY) { - auto new_child = MaxLogicalType(ArrayType::GetChildType(left), ArrayType::GetChildType(right)); + case LogicalTypeId::ARRAY: { + LogicalType new_child; + if (!OP::Operation(ArrayType::GetChildType(left), ArrayType::GetChildType(right), new_child)) { + return false; + } auto new_size = MaxValue(ArrayType::GetSize(left), ArrayType::GetSize(right)); - return LogicalType::ARRAY(new_child, new_size); + result = LogicalType::ARRAY(new_child, new_size); + return true; } - if (type_id == LogicalTypeId::MAP) { - // list: perform max recursively on child type - auto new_child = MaxLogicalType(ListType::GetChildType(left), ListType::GetChildType(right)); - return LogicalType::MAP(new_child); + case LogicalTypeId::MAP: { + // map: perform max recursively on child type + LogicalType new_child; + if (!OP::Operation(ListType::GetChildType(left), ListType::GetChildType(right), new_child)) { + return false; + } + result = LogicalType::MAP(new_child); + return true; } - if (type_id == LogicalTypeId::STRUCT) { - // struct: perform recursively + case LogicalTypeId::STRUCT: { + // struct: perform recursively on each child auto &left_child_types = StructType::GetChildTypes(left); auto &right_child_types = StructType::GetChildTypes(right); if (left_child_types.size() != right_child_types.size()) { - // child types are not of equal size, we can't cast anyway - // just return the left child - return left; + // child types are not of equal size, we can't cast + // return false + return false; } child_list_t child_types; for (idx_t i = 0; i < left_child_types.size(); i++) { - auto child_type = MaxLogicalType(left_child_types[i].second, right_child_types[i].second); + LogicalType child_type; + if (!OP::Operation(left_child_types[i].second, right_child_types[i].second, child_type)) { + return false; + } child_types.emplace_back(left_child_types[i].first, std::move(child_type)); } - - return LogicalType::STRUCT(child_types); + result = LogicalType::STRUCT(child_types); + return true; } - if (type_id == LogicalTypeId::UNION) { + case LogicalTypeId::UNION: { auto left_member_count = UnionType::GetMemberCount(left); auto right_member_count = UnionType::GetMemberCount(right); if (left_member_count != right_member_count) { // return the "larger" type, with the most members - return left_member_count > right_member_count ? left : right; + result = left_member_count > right_member_count ? left : right; + return true; } // otherwise, keep left, don't try to meld the two together. + result = left; + return true; + } + default: + result = left; + return true; + } +} + +template +bool TryGetMaxLogicalTypeInternal(const LogicalType &left, const LogicalType &right, LogicalType &result) { + // we always prefer aliased types + if (!left.GetAlias().empty()) { + result = left; + return true; + } + if (!right.GetAlias().empty()) { + result = right; + return true; + } + if (left.id() != right.id()) { + return CombineUnequalTypes(left, right, result); + } else { + return CombineEqualTypes(left, right, result); + } +} + +struct TryGetTypeOperation { + static bool Operation(const LogicalType &left, const LogicalType &right, LogicalType &result) { + return TryGetMaxLogicalTypeInternal(left, right, result); + } +}; + +struct ForceGetTypeOperation { + static bool Operation(const LogicalType &left, const LogicalType &right, LogicalType &result) { + result = LogicalType::ForceMaxLogicalType(left, right); + return true; + } +}; + +bool LogicalType::TryGetMaxLogicalType(ClientContext &context, const LogicalType &left, const LogicalType &right, + LogicalType &result) { + if (DBConfig::GetConfig(context).options.old_implicit_casting) { + result = LogicalType::ForceMaxLogicalType(left, right); + return true; + } + return TryGetMaxLogicalTypeInternal(left, right, result); +} + +static idx_t GetLogicalTypeScore(const LogicalType &type) { + switch (type.id()) { + case LogicalTypeId::INVALID: + case LogicalTypeId::SQLNULL: + case LogicalTypeId::UNKNOWN: + case LogicalTypeId::ANY: + case LogicalTypeId::STRING_LITERAL: + case LogicalTypeId::INTEGER_LITERAL: + return 0; + // numerics + case LogicalTypeId::BOOLEAN: + return 10; + case LogicalTypeId::UTINYINT: + return 11; + case LogicalTypeId::TINYINT: + return 12; + case LogicalTypeId::USMALLINT: + return 13; + case LogicalTypeId::SMALLINT: + return 14; + case LogicalTypeId::UINTEGER: + return 15; + case LogicalTypeId::INTEGER: + return 16; + case LogicalTypeId::UBIGINT: + return 17; + case LogicalTypeId::BIGINT: + return 18; + case LogicalTypeId::UHUGEINT: + return 19; + case LogicalTypeId::HUGEINT: + return 20; + case LogicalTypeId::DECIMAL: + return 21; + case LogicalTypeId::FLOAT: + return 22; + case LogicalTypeId::DOUBLE: + return 23; + // date/time/timestamp + case LogicalTypeId::TIME: + case LogicalTypeId::TIME_TZ: + return 50; + case LogicalTypeId::DATE: + return 51; + case LogicalTypeId::TIMESTAMP_SEC: + return 52; + case LogicalTypeId::TIMESTAMP_MS: + return 53; + case LogicalTypeId::TIMESTAMP: + case LogicalTypeId::TIMESTAMP_TZ: + return 54; + case LogicalTypeId::TIMESTAMP_NS: + return 55; + case LogicalTypeId::INTERVAL: + return 56; + // text/character strings + case LogicalTypeId::CHAR: + return 75; + case LogicalTypeId::VARCHAR: + return 77; + case LogicalTypeId::ENUM: + return 78; + // blob/complex types + case LogicalTypeId::BIT: + return 100; + case LogicalTypeId::BLOB: + return 101; + case LogicalTypeId::UUID: + return 102; + // nested types + case LogicalTypeId::STRUCT: + return 125; + case LogicalTypeId::LIST: + case LogicalTypeId::ARRAY: + return 126; + case LogicalTypeId::MAP: + return 127; + case LogicalTypeId::UNION: + case LogicalTypeId::TABLE: + return 150; + // weirdo types + case LogicalTypeId::LAMBDA: + case LogicalTypeId::AGGREGATE_STATE: + case LogicalTypeId::POINTER: + case LogicalTypeId::VALIDITY: + case LogicalTypeId::USER: + break; + } + return 1000; +} + +LogicalType LogicalType::ForceMaxLogicalType(const LogicalType &left, const LogicalType &right) { + LogicalType result; + if (TryGetMaxLogicalTypeInternal(left, right, result)) { + return result; + } + // we prefer the type with the highest score + auto left_score = GetLogicalTypeScore(left); + auto right_score = GetLogicalTypeScore(right); + if (left_score < right_score) { + return right; + } else { return left; } - // types are equal but no extra specifier: just return the type - return left; +} + +LogicalType LogicalType::MaxLogicalType(ClientContext &context, const LogicalType &left, const LogicalType &right) { + LogicalType result; + if (!TryGetMaxLogicalType(context, left, right, result)) { + throw NotImplementedException("Cannot combine types %s and %s - an explicit cast is required", left.ToString(), + right.ToString()); + } + return result; } void LogicalType::Verify() const { @@ -960,12 +1223,24 @@ const string &StructType::GetChildName(const LogicalType &type, idx_t index) { return child_types[index].first; } +idx_t StructType::GetChildIndexUnsafe(const LogicalType &type, const string &name) { + auto &child_types = StructType::GetChildTypes(type); + for (idx_t i = 0; i < child_types.size(); i++) { + if (StringUtil::CIEquals(child_types[i].first, name)) { + return i; + } + } + throw InternalException("Could not find child with name \"%s\" in struct type \"%s\"", name, type.ToString()); +} + idx_t StructType::GetChildCount(const LogicalType &type) { return StructType::GetChildTypes(type).size(); } bool StructType::IsUnnamed(const LogicalType &type) { auto &child_types = StructType::GetChildTypes(type); - D_ASSERT(child_types.size() > 0); + if (child_types.empty()) { + return false; + } return child_types[0].first.empty(); } @@ -1131,6 +1406,19 @@ PhysicalType EnumType::GetPhysicalType(const LogicalType &type) { return EnumTypeInfo::DictType(info.GetDictSize()); } +//===--------------------------------------------------------------------===// +// JSON Type +//===--------------------------------------------------------------------===// +LogicalType LogicalType::JSON() { + auto json_type = LogicalType(LogicalTypeId::VARCHAR); + json_type.SetAlias(JSON_TYPE_NAME); + return json_type; +} + +bool LogicalType::IsJSONType() const { + return id() == LogicalTypeId::VARCHAR && HasAlias() && GetAlias() == JSON_TYPE_NAME; +} + //===--------------------------------------------------------------------===// // Array Type //===--------------------------------------------------------------------===// @@ -1168,6 +1456,67 @@ LogicalType LogicalType::ARRAY(const LogicalType &child) { return LogicalType(LogicalTypeId::ARRAY, std::move(info)); } +//===--------------------------------------------------------------------===// +// Any Type +//===--------------------------------------------------------------------===// +LogicalType LogicalType::ANY_PARAMS(LogicalType target, idx_t cast_score) { // NOLINT + auto type_info = make_shared(std::move(target), cast_score); + return LogicalType(LogicalTypeId::ANY, std::move(type_info)); +} + +LogicalType AnyType::GetTargetType(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::ANY); + auto info = type.AuxInfo(); + if (!info) { + return LogicalType::ANY; + } + return info->Cast().target_type; +} + +idx_t AnyType::GetCastScore(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::ANY); + auto info = type.AuxInfo(); + if (!info) { + return 5; + } + return info->Cast().cast_score; +} + +//===--------------------------------------------------------------------===// +// Integer Literal Type +//===--------------------------------------------------------------------===// +LogicalType IntegerLiteral::GetType(const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::INTEGER_LITERAL); + auto info = type.AuxInfo(); + D_ASSERT(info && info->type == ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO); + return info->Cast().constant_value.type(); +} + +bool IntegerLiteral::FitsInType(const LogicalType &type, const LogicalType &target) { + D_ASSERT(type.id() == LogicalTypeId::INTEGER_LITERAL); + // we can always cast integer literals to float and double + if (target.id() == LogicalTypeId::FLOAT || target.id() == LogicalTypeId::DOUBLE) { + return true; + } + if (!target.IsIntegral()) { + return false; + } + // we can cast to integral types if the constant value fits within that type + auto info = type.AuxInfo(); + D_ASSERT(info && info->type == ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO); + auto &literal_info = info->Cast(); + Value copy = literal_info.constant_value; + return copy.DefaultTryCastAs(target); +} + +LogicalType LogicalType::INTEGER_LITERAL(const Value &constant) { // NOLINT + if (!constant.type().IsIntegral()) { + throw InternalException("INTEGER_LITERAL can only be made from literals of integer types"); + } + auto type_info = make_shared(constant); + return LogicalType(LogicalTypeId::INTEGER_LITERAL, std::move(type_info)); +} + //===--------------------------------------------------------------------===// // Logical Type //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/common/types/cast_helpers.cpp b/src/duckdb/src/common/types/cast_helpers.cpp index 5011b674b..f37fbaa97 100644 --- a/src/duckdb/src/common/types/cast_helpers.cpp +++ b/src/duckdb/src/common/types/cast_helpers.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/types/cast_helpers.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" namespace duckdb { @@ -107,4 +108,9 @@ std::string NumericHelper::ToString(hugeint_t value) { return Hugeint::ToString(value); } +template <> +std::string NumericHelper::ToString(uhugeint_t value) { + return Uhugeint::ToString(value); +} + } // namespace duckdb diff --git a/src/duckdb/src/common/types/chunk_collection.cpp b/src/duckdb/src/common/types/chunk_collection.cpp deleted file mode 100644 index abd4314e3..000000000 --- a/src/duckdb/src/common/types/chunk_collection.cpp +++ /dev/null @@ -1,190 +0,0 @@ -#include "duckdb/common/types/chunk_collection.hpp" - -#include "duckdb/common/assert.hpp" -#include "duckdb/common/exception.hpp" -#include "duckdb/common/queue.hpp" -#include "duckdb/common/operator/comparison_operators.hpp" -#include "duckdb/common/printer.hpp" -#include "duckdb/common/value_operations/value_operations.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" - -#include -#include - -namespace duckdb { - -ChunkCollection::ChunkCollection(Allocator &allocator) : allocator(allocator), count(0) { -} - -ChunkCollection::ChunkCollection(ClientContext &context) : ChunkCollection(Allocator::Get(context)) { -} - -void ChunkCollection::Verify() { -#ifdef DEBUG - for (auto &chunk : chunks) { - chunk->Verify(); - } -#endif -} - -void ChunkCollection::Append(ChunkCollection &other) { - for (auto &chunk : other.chunks) { - Append(*chunk); - } -} - -void ChunkCollection::Merge(ChunkCollection &other) { - if (other.count == 0) { - return; - } - if (count == 0) { - chunks = std::move(other.chunks); - types = std::move(other.types); - count = other.count; - return; - } - unique_ptr old_back; - if (!chunks.empty() && chunks.back()->size() != STANDARD_VECTOR_SIZE) { - old_back = std::move(chunks.back()); - chunks.pop_back(); - count -= old_back->size(); - } - for (auto &chunk : other.chunks) { - chunks.push_back(std::move(chunk)); - } - count += other.count; - if (old_back) { - Append(*old_back); - } - Verify(); -} - -void ChunkCollection::Append(DataChunk &new_chunk) { - if (new_chunk.size() == 0) { - return; - } - new_chunk.Verify(); - - // we have to ensure that every chunk in the ChunkCollection is completely - // filled, otherwise our O(1) lookup in GetValue and SetValue does not work - // first fill the latest chunk, if it exists - count += new_chunk.size(); - - idx_t remaining_data = new_chunk.size(); - idx_t offset = 0; - if (chunks.empty()) { - // first chunk - types = new_chunk.GetTypes(); - } else { - // the types of the new chunk should match the types of the previous one - D_ASSERT(types.size() == new_chunk.ColumnCount()); - auto new_types = new_chunk.GetTypes(); - for (idx_t i = 0; i < types.size(); i++) { - if (new_types[i] != types[i]) { - throw TypeMismatchException(new_types[i], types[i], "Type mismatch when combining rows"); - } - if (types[i].InternalType() == PhysicalType::LIST) { - // need to check all the chunks because they can have only-null list entries - for (auto &chunk : chunks) { - auto &chunk_vec = chunk->data[i]; - auto &new_vec = new_chunk.data[i]; - auto &chunk_type = chunk_vec.GetType(); - auto &new_type = new_vec.GetType(); - if (chunk_type != new_type) { - throw TypeMismatchException(chunk_type, new_type, "Type mismatch when combining lists"); - } - } - } - // TODO check structs, too - } - - // first append data to the current chunk - DataChunk &last_chunk = *chunks.back(); - idx_t added_data = MinValue(remaining_data, STANDARD_VECTOR_SIZE - last_chunk.size()); - if (added_data > 0) { - // copy elements to the last chunk - new_chunk.Flatten(); - // have to be careful here: setting the cardinality without calling normalify can cause incorrect partial - // decompression - idx_t old_count = new_chunk.size(); - new_chunk.SetCardinality(added_data); - - last_chunk.Append(new_chunk); - remaining_data -= added_data; - // reset the chunk to the old data - new_chunk.SetCardinality(old_count); - offset = added_data; - } - } - - if (remaining_data > 0) { - // create a new chunk and fill it with the remainder - auto chunk = make_uniq(); - chunk->Initialize(allocator, types); - new_chunk.Copy(*chunk, offset); - chunks.push_back(std::move(chunk)); - } -} - -void ChunkCollection::Append(unique_ptr new_chunk) { - if (types.empty()) { - types = new_chunk->GetTypes(); - } - D_ASSERT(types == new_chunk->GetTypes()); - count += new_chunk->size(); - chunks.push_back(std::move(new_chunk)); -} - -void ChunkCollection::Fuse(ChunkCollection &other) { - if (count == 0) { - chunks.reserve(other.ChunkCount()); - for (idx_t chunk_idx = 0; chunk_idx < other.ChunkCount(); ++chunk_idx) { - auto lhs = make_uniq(); - auto &rhs = other.GetChunk(chunk_idx); - lhs->data.reserve(rhs.data.size()); - for (auto &v : rhs.data) { - lhs->data.emplace_back(v); - } - lhs->SetCardinality(rhs.size()); - chunks.push_back(std::move(lhs)); - } - count = other.Count(); - } else { - D_ASSERT(this->ChunkCount() == other.ChunkCount()); - for (idx_t chunk_idx = 0; chunk_idx < ChunkCount(); ++chunk_idx) { - auto &lhs = this->GetChunk(chunk_idx); - auto &rhs = other.GetChunk(chunk_idx); - D_ASSERT(lhs.size() == rhs.size()); - for (auto &v : rhs.data) { - lhs.data.emplace_back(v); - } - } - } - types.insert(types.end(), other.types.begin(), other.types.end()); -} - -Value ChunkCollection::GetValue(idx_t column, idx_t index) { - return chunks[LocateChunk(index)]->GetValue(column, index % STANDARD_VECTOR_SIZE); -} - -void ChunkCollection::SetValue(idx_t column, idx_t index, const Value &value) { - chunks[LocateChunk(index)]->SetValue(column, index % STANDARD_VECTOR_SIZE, value); -} - -void ChunkCollection::CopyCell(idx_t column, idx_t index, Vector &target, idx_t target_offset) { - auto &chunk = GetChunkForRow(index); - auto &source = chunk.data[column]; - const auto source_offset = index % STANDARD_VECTOR_SIZE; - VectorOperations::Copy(source, target, source_offset + 1, source_offset, target_offset); -} - -string ChunkCollection::ToString() const { - return chunks.empty() ? "ChunkCollection [ 0 ]" - : "ChunkCollection [ " + std::to_string(count) + " ]: \n" + chunks[0]->ToString(); -} - -void ChunkCollection::Print() const { - Printer::Print(ToString()); -} - -} // namespace duckdb 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 de135432a..9305c68f2 100644 --- a/src/duckdb/src/common/types/column/column_data_collection.cpp +++ b/src/duckdb/src/common/types/column/column_data_collection.cpp @@ -4,6 +4,7 @@ #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/column/column_data_collection_segment.hpp" #include "duckdb/common/types/value_map.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/common/serializer/serializer.hpp" @@ -717,6 +718,9 @@ ColumnDataCopyFunction ColumnDataCollection::GetCopyFunction(const LogicalType & case PhysicalType::UINT64: function = ColumnDataCopy; break; + case PhysicalType::UINT128: + function = ColumnDataCopy; + break; case PhysicalType::FLOAT: function = ColumnDataCopy; break; diff --git a/src/duckdb/src/common/types/date.cpp b/src/duckdb/src/common/types/date.cpp index d1e2679eb..39110c9fe 100644 --- a/src/duckdb/src/common/types/date.cpp +++ b/src/duckdb/src/common/types/date.cpp @@ -6,8 +6,8 @@ #include "duckdb/common/string_util.hpp" #include "duckdb/common/assert.hpp" #include "duckdb/common/operator/multiply.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" #include "duckdb/common/limits.hpp" - #include #include #include diff --git a/src/duckdb/src/common/types/hash.cpp b/src/duckdb/src/common/types/hash.cpp index 1e069d366..c3136a526 100644 --- a/src/duckdb/src/common/types/hash.cpp +++ b/src/duckdb/src/common/types/hash.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/helper.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/types/interval.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include #include @@ -24,6 +25,11 @@ hash_t Hash(hugeint_t val) { return murmurhash64(val.lower) ^ murmurhash64(val.upper); } +template <> +hash_t Hash(uhugeint_t val) { + return murmurhash64(val.lower) ^ murmurhash64(val.upper); +} + template struct FloatingPointEqualityTransform { static void OP(T &val) { diff --git a/src/duckdb/src/common/types/hugeint.cpp b/src/duckdb/src/common/types/hugeint.cpp index c700b46b7..0ef7e0fc5 100644 --- a/src/duckdb/src/common/types/hugeint.cpp +++ b/src/duckdb/src/common/types/hugeint.cpp @@ -1,4 +1,5 @@ #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/algorithm.hpp" #include "duckdb/common/hugeint.hpp" @@ -56,6 +57,29 @@ const hugeint_t Hugeint::POWERS_OF_TEN[] { hugeint_t(1000000000000000000) * hugeint_t(1000000000000000000) * hugeint_t(10), hugeint_t(1000000000000000000) * hugeint_t(1000000000000000000) * hugeint_t(100)}; +//===--------------------------------------------------------------------===// +// Negate +//===--------------------------------------------------------------------===// + +template <> +void Hugeint::NegateInPlace(hugeint_t &input) { + input.lower = NumericLimits::Maximum() - input.lower + 1; + input.upper = -1 - input.upper + (input.lower == 0); +} + +bool Hugeint::TryNegate(hugeint_t input, hugeint_t &result) { + if (input.upper == NumericLimits::Minimum() && input.lower == 0) { + return false; + } + NegateInPlace(input); + result = input; + return true; +} + +//===--------------------------------------------------------------------===// +// Divide +//===--------------------------------------------------------------------===// + static uint8_t PositiveHugeintHighestBit(hugeint_t bits) { uint8_t out = 0; if (bits.upper) { @@ -83,7 +107,7 @@ static bool PositiveHugeintIsBitSet(hugeint_t lhs, uint8_t bit_position) { } } -hugeint_t PositiveHugeintLeftShift(hugeint_t lhs, uint32_t amount) { +static hugeint_t PositiveHugeintLeftShift(hugeint_t lhs, uint32_t amount) { D_ASSERT(amount > 0 && amount < 64); hugeint_t result; result.lower = lhs.lower << amount; @@ -153,10 +177,16 @@ string Hugeint::ToString(hugeint_t input) { //===--------------------------------------------------------------------===// // Multiply //===--------------------------------------------------------------------===// + +// Multiply with overflow checks bool Hugeint::TryMultiply(hugeint_t lhs, hugeint_t rhs, hugeint_t &result) { // Check if one of the sides is hugeint_t minimum, as that can't be negated. - // You can only multiply the minimum by 1, any other value will result in overflow + // You can only multiply the minimum by 0 or 1, any other value will result in overflow if (lhs == NumericLimits::Minimum() || rhs == NumericLimits::Minimum()) { + if (lhs == 0 || rhs == 0) { + result = 0; + return true; + } if (lhs == 1 || rhs == 1) { result = NumericLimits::Minimum(); return true; @@ -166,12 +196,13 @@ bool Hugeint::TryMultiply(hugeint_t lhs, hugeint_t rhs, hugeint_t &result) { bool lhs_negative = lhs.upper < 0; bool rhs_negative = rhs.upper < 0; - if (lhs_negative) { - NegateInPlace(lhs); + if (lhs_negative && !TryNegate(lhs, lhs)) { + return false; } - if (rhs_negative) { - NegateInPlace(rhs); + if (rhs_negative && !TryNegate(rhs, rhs)) { + return false; } + #if ((__GNUC__ >= 5) || defined(__clang__)) && defined(__SIZEOF_INT128__) __uint128_t left = __uint128_t(lhs.lower) + (__uint128_t(lhs.upper) << 64); __uint128_t right = __uint128_t(rhs.lower) + (__uint128_t(rhs.upper) << 64); @@ -254,15 +285,85 @@ bool Hugeint::TryMultiply(hugeint_t lhs, hugeint_t rhs, hugeint_t &result) { result.upper = (first32 << 32) | second32; #endif if (lhs_negative ^ rhs_negative) { - NegateInPlace(result); + NegateInPlace(result); } return true; } -hugeint_t Hugeint::Multiply(hugeint_t lhs, hugeint_t rhs) { +// Multiply without overflow check +template <> +hugeint_t Hugeint::Multiply(hugeint_t lhs, hugeint_t rhs) { hugeint_t result; - if (!TryMultiply(lhs, rhs, result)) { - throw OutOfRangeException("Overflow in HUGEINT multiplication!"); + bool lhs_negative = lhs.upper < 0; + bool rhs_negative = rhs.upper < 0; + if (lhs_negative) { + NegateInPlace(lhs); + } + if (rhs_negative) { + NegateInPlace(rhs); + } + +#if ((__GNUC__ >= 5) || defined(__clang__)) && defined(__SIZEOF_INT128__) + __uint128_t left = __uint128_t(lhs.lower) + (__uint128_t(lhs.upper) << 64); + __uint128_t right = __uint128_t(rhs.lower) + (__uint128_t(rhs.upper) << 64); + __uint128_t result_i128; + result_i128 = left * right; + uint64_t upper = uint64_t(result_i128 >> 64); + result.upper = int64_t(upper); + result.lower = uint64_t(result_i128 & 0xffffffffffffffff); +#else + // Multiply code adapted from: + // https://github.com/calccrypto/uint128_t/blob/master/uint128_t.cpp + + // split values into 4 32-bit parts + uint64_t top[4] = {uint64_t(lhs.upper) >> 32, uint64_t(lhs.upper) & 0xffffffff, lhs.lower >> 32, + lhs.lower & 0xffffffff}; + uint64_t bottom[4] = {uint64_t(rhs.upper) >> 32, uint64_t(rhs.upper) & 0xffffffff, rhs.lower >> 32, + rhs.lower & 0xffffffff}; + uint64_t products[4][4]; + + // multiply each component of the values + for (auto x = 0; x < 4; x++) { + for (auto y = 0; y < 4; y++) { + products[x][y] = top[x] * bottom[y]; + } + } + + // first row + uint64_t fourth32 = (products[3][3] & 0xffffffff); + uint64_t third32 = (products[3][2] & 0xffffffff) + (products[3][3] >> 32); + uint64_t second32 = (products[3][1] & 0xffffffff) + (products[3][2] >> 32); + uint64_t first32 = (products[3][0] & 0xffffffff) + (products[3][1] >> 32); + + // second row + third32 += (products[2][3] & 0xffffffff); + second32 += (products[2][2] & 0xffffffff) + (products[2][3] >> 32); + first32 += (products[2][1] & 0xffffffff) + (products[2][2] >> 32); + + // third row + second32 += (products[1][3] & 0xffffffff); + first32 += (products[1][2] & 0xffffffff) + (products[1][3] >> 32); + + // fourth row + first32 += (products[0][3] & 0xffffffff); + + // move carry to next digit + third32 += fourth32 >> 32; + second32 += third32 >> 32; + first32 += second32 >> 32; + + // remove carry from current digit + fourth32 &= 0xffffffff; + third32 &= 0xffffffff; + second32 &= 0xffffffff; + first32 &= 0xffffffff; + + // combine components + result.lower = (third32 << 32) | fourth32; + result.upper = (first32 << 32) | second32; +#endif + if (lhs_negative ^ rhs_negative) { + NegateInPlace(result); } return result; } @@ -271,13 +372,13 @@ hugeint_t Hugeint::Multiply(hugeint_t lhs, hugeint_t rhs) { // Divide //===--------------------------------------------------------------------===// -static hugeint_t Sign(hugeint_t n) { +int Sign(hugeint_t n) { return ((n > 0) - (n < 0)); } -static hugeint_t Abs(hugeint_t n) { +hugeint_t Abs(hugeint_t n) { D_ASSERT(n != NumericLimits::Minimum()); - return (Sign(n) * n); + return (n * Sign(n)); } static hugeint_t DivModMinimum(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainder) { @@ -291,11 +392,6 @@ static hugeint_t DivModMinimum(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainde return 0; } - if (rhs == -1) { - throw OutOfRangeException("Overflow in division of INT128 (%s // %s)!", lhs.ToString().c_str(), - rhs.ToString().c_str()); - } - // Add 1 to minimum and run through DivMod again hugeint_t result = Hugeint::DivMod(NumericLimits::Minimum() + 1, rhs, remainder); @@ -309,9 +405,12 @@ static hugeint_t DivModMinimum(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainde return result; } +// No overflow checks hugeint_t Hugeint::DivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainder) { - // division by zero not allowed - D_ASSERT(!(rhs.upper == 0 && rhs.lower == 0)); + if (rhs == 0) { + remainder = lhs; + return hugeint_t(0); + } // Check if one of the sides is hugeint_t minimum, as that can't be negated. if (lhs == NumericLimits::Minimum() || rhs == NumericLimits::Minimum()) { @@ -321,10 +420,10 @@ hugeint_t Hugeint::DivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainder) { bool lhs_negative = lhs.upper < 0; bool rhs_negative = rhs.upper < 0; if (lhs_negative) { - Hugeint::NegateInPlace(lhs); + Hugeint::NegateInPlace(lhs); } if (rhs_negative) { - Hugeint::NegateInPlace(rhs); + Hugeint::NegateInPlace(rhs); } // DivMod code adapted from: // https://github.com/calccrypto/uint128_t/blob/master/uint128_t.cpp @@ -345,39 +444,55 @@ hugeint_t Hugeint::DivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainder) { // we get the value of the bit at position X, where position 0 is the least-significant bit if (PositiveHugeintIsBitSet(lhs, x - 1)) { - // increment the remainder - Hugeint::AddInPlace(remainder, 1); + remainder += 1; } if (Hugeint::GreaterThanEquals(remainder, rhs)) { // the remainder has passed the division multiplier: add one to the divide result - remainder = Hugeint::Subtract(remainder, rhs); - Hugeint::AddInPlace(div_result, 1); + remainder -= rhs; + div_result += 1; } } if (lhs_negative ^ rhs_negative) { - Hugeint::NegateInPlace(div_result); + Hugeint::NegateInPlace(div_result); } if (lhs_negative) { - Hugeint::NegateInPlace(remainder); + Hugeint::NegateInPlace(remainder); } return div_result; } -hugeint_t Hugeint::Divide(hugeint_t lhs, hugeint_t rhs) { +bool Hugeint::TryDivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &result, hugeint_t &remainder) { + // No division by zero + if (rhs == 0) { + return false; + } + + // division only has one reason to overflow: MINIMUM / -1 + if (lhs == NumericLimits::Minimum() && rhs == -1) { + return false; + } + + result = Hugeint::DivMod(lhs, rhs, remainder); + return true; +} + +template <> +hugeint_t Hugeint::Divide(hugeint_t lhs, hugeint_t rhs) { hugeint_t remainder; return Hugeint::DivMod(lhs, rhs, remainder); } -hugeint_t Hugeint::Modulo(hugeint_t lhs, hugeint_t rhs) { +template <> +hugeint_t Hugeint::Modulo(hugeint_t lhs, hugeint_t rhs) { hugeint_t remainder; - Hugeint::DivMod(lhs, rhs, remainder); + (void)Hugeint::DivMod(lhs, rhs, remainder); return remainder; } //===--------------------------------------------------------------------===// // Add/Subtract //===--------------------------------------------------------------------===// -bool Hugeint::AddInPlace(hugeint_t &lhs, hugeint_t rhs) { +bool Hugeint::TryAddInPlace(hugeint_t &lhs, hugeint_t rhs) { int overflow = lhs.lower + rhs.lower < lhs.lower; if (rhs.upper >= 0) { // RHS is positive: check for overflow @@ -396,7 +511,7 @@ bool Hugeint::AddInPlace(hugeint_t &lhs, hugeint_t rhs) { return true; } -bool Hugeint::SubtractInPlace(hugeint_t &lhs, hugeint_t rhs) { +bool Hugeint::TrySubtractInPlace(hugeint_t &lhs, hugeint_t rhs) { // underflow int underflow = lhs.lower - rhs.lower > lhs.lower; if (rhs.upper >= 0) { @@ -417,18 +532,14 @@ bool Hugeint::SubtractInPlace(hugeint_t &lhs, hugeint_t rhs) { return true; } -hugeint_t Hugeint::Add(hugeint_t lhs, hugeint_t rhs) { - if (!AddInPlace(lhs, rhs)) { - throw OutOfRangeException("Overflow in HUGEINT addition: %s + %s", lhs.ToString(), rhs.ToString()); - } - return lhs; +template <> +hugeint_t Hugeint::Add(hugeint_t lhs, hugeint_t rhs) { + return lhs + rhs; } -hugeint_t Hugeint::Subtract(hugeint_t lhs, hugeint_t rhs) { - if (!SubtractInPlace(lhs, rhs)) { - throw OutOfRangeException("Underflow in HUGEINT addition: %s - %s", lhs.ToString(), rhs.ToString()); - } - return lhs; +template <> +hugeint_t Hugeint::Subtract(hugeint_t lhs, hugeint_t rhs) { + return lhs - rhs; } //===--------------------------------------------------------------------===// @@ -506,6 +617,17 @@ bool Hugeint::TryCast(hugeint_t input, hugeint_t &result) { return true; } +template <> +bool Hugeint::TryCast(hugeint_t input, uhugeint_t &result) { + if (input < 0) { + return false; + } + + result.lower = input.lower; + result.upper = input.upper; + return true; +} + template <> bool Hugeint::TryCast(hugeint_t input, float &result) { double dbl_result; @@ -672,27 +794,29 @@ bool hugeint_t::operator>=(const hugeint_t &rhs) const { } hugeint_t hugeint_t::operator+(const hugeint_t &rhs) const { - return Hugeint::Add(*this, rhs); + return hugeint_t(upper + rhs.upper + ((lower + rhs.lower) < lower), lower + rhs.lower); } hugeint_t hugeint_t::operator-(const hugeint_t &rhs) const { - return Hugeint::Subtract(*this, rhs); + return hugeint_t(upper - rhs.upper - ((lower - rhs.lower) > lower), lower - rhs.lower); } hugeint_t hugeint_t::operator*(const hugeint_t &rhs) const { - return Hugeint::Multiply(*this, rhs); + hugeint_t result = *this; + result *= rhs; + return result; } hugeint_t hugeint_t::operator/(const hugeint_t &rhs) const { - return Hugeint::Divide(*this, rhs); + return Hugeint::Divide(*this, rhs); } hugeint_t hugeint_t::operator%(const hugeint_t &rhs) const { - return Hugeint::Modulo(*this, rhs); + return Hugeint::Modulo(*this, rhs); } hugeint_t hugeint_t::operator-() const { - return Hugeint::Negate(*this); + return Hugeint::Negate(*this); } hugeint_t hugeint_t::operator>>(const hugeint_t &rhs) const { @@ -772,23 +896,23 @@ hugeint_t hugeint_t::operator~() const { } hugeint_t &hugeint_t::operator+=(const hugeint_t &rhs) { - Hugeint::AddInPlace(*this, rhs); + *this = *this + rhs; return *this; } hugeint_t &hugeint_t::operator-=(const hugeint_t &rhs) { - Hugeint::SubtractInPlace(*this, rhs); + *this = *this - rhs; return *this; } hugeint_t &hugeint_t::operator*=(const hugeint_t &rhs) { - *this = Hugeint::Multiply(*this, rhs); + *this = Hugeint::Multiply(*this, rhs); return *this; } hugeint_t &hugeint_t::operator/=(const hugeint_t &rhs) { - *this = Hugeint::Divide(*this, rhs); + *this = Hugeint::Divide(*this, rhs); return *this; } hugeint_t &hugeint_t::operator%=(const hugeint_t &rhs) { - *this = Hugeint::Modulo(*this, rhs); + *this = Hugeint::Modulo(*this, rhs); return *this; } hugeint_t &hugeint_t::operator>>=(const hugeint_t &rhs) { @@ -853,6 +977,9 @@ hugeint_t::operator int32_t() const { hugeint_t::operator int64_t() const { return NarrowCast(*this); } +hugeint_t::operator uhugeint_t() const { + return {static_cast(this->upper), this->lower}; +} string hugeint_t::ToString() const { return Hugeint::ToString(*this); diff --git a/src/duckdb/src/common/types/hyperloglog.cpp b/src/duckdb/src/common/types/hyperloglog.cpp index 4efcdc820..0b568ce00 100644 --- a/src/duckdb/src/common/types/hyperloglog.cpp +++ b/src/duckdb/src/common/types/hyperloglog.cpp @@ -54,7 +54,7 @@ HyperLogLog *HyperLogLog::MergePointer(HyperLogLog &other) { hlls[1] = other.hll; auto new_hll = duckdb_hll::hll_merge(hlls, 2); if (!new_hll) { - throw Exception("Could not merge HLLs"); + throw InternalException("Could not merge HLLs"); } return new HyperLogLog(new_hll); } @@ -129,6 +129,12 @@ inline uint64_t TemplatedHash(const hugeint_t &elem) { TemplatedHash(elem.lower); } +template <> +inline uint64_t TemplatedHash(const uhugeint_t &elem) { + return TemplatedHash(Load(const_data_ptr_cast(&elem.upper))) ^ + TemplatedHash(elem.lower); +} + template inline void CreateIntegerRecursive(const_data_ptr_t &data, uint64_t &x) { x ^= (uint64_t)data[rest - 1] << ((rest - 1) * 8); @@ -226,9 +232,10 @@ static void ComputeHashes(UnifiedVectorFormat &vdata, const LogicalType &type, u case PhysicalType::DOUBLE: return TemplatedComputeHashes(vdata, count, hashes); case PhysicalType::INT128: + case PhysicalType::UINT128: case PhysicalType::INTERVAL: - static_assert(sizeof(hugeint_t) == sizeof(interval_t), "ComputeHashes assumes these are the same size!"); - return TemplatedComputeHashes(vdata, count, hashes); + static_assert(sizeof(uhugeint_t) == sizeof(interval_t), "ComputeHashes assumes these are the same size!"); + return TemplatedComputeHashes(vdata, count, hashes); case PhysicalType::VARCHAR: return TemplatedComputeHashes(vdata, count, hashes); default: diff --git a/src/duckdb/src/common/types/interval.cpp b/src/duckdb/src/common/types/interval.cpp index e3eff43f1..569f89c06 100644 --- a/src/duckdb/src/common/types/interval.cpp +++ b/src/duckdb/src/common/types/interval.cpp @@ -22,7 +22,7 @@ bool Interval::FromString(const string &str, interval_t &result) { } template -void IntervalTryAddition(T &target, int64_t input, int64_t multiplier) { +void IntervalTryAddition(T &target, int64_t input, int64_t multiplier, int64_t fraction = 0) { int64_t addition; if (!TryMultiplyOperator::Operation(input, multiplier, addition)) { throw OutOfRangeException("interval value is out of range"); @@ -31,6 +31,15 @@ void IntervalTryAddition(T &target, int64_t input, int64_t multiplier) { if (!TryAddOperator::Operation(target, addition_base, target)) { throw OutOfRangeException("interval value is out of range"); } + if (fraction) { + // Add in (fraction * multiplier) / MICROS_PER_SEC + // This is always in range + addition = (fraction * multiplier) / Interval::MICROS_PER_SEC; + addition_base = Cast::Operation(addition); + if (!TryAddOperator::Operation(target, addition_base, target)) { + throw OutOfRangeException("interval fraction is out of range"); + } + } } bool Interval::FromCString(const char *str, idx_t len, interval_t &result, string *error_message, bool strict) { @@ -110,7 +119,7 @@ bool Interval::FromCString(const char *str, idx_t len, interval_t &result, strin int32_t mult = 100000; for (++pos; pos < len && StringUtil::CharacterIsDigit(str[pos]); ++pos, mult /= 10) { if (mult > 0) { - fraction += (str[pos] - '0') * mult; + fraction += int64_t(str[pos] - '0') * mult; } } } @@ -184,43 +193,55 @@ interval_parse_time : { // add the specifier to the interval switch (specifier) { case DatePartSpecifier::MILLENNIUM: - IntervalTryAddition(result.months, number, MONTHS_PER_MILLENIUM); + IntervalTryAddition(result.months, number, MONTHS_PER_MILLENIUM, fraction); break; case DatePartSpecifier::CENTURY: - IntervalTryAddition(result.months, number, MONTHS_PER_CENTURY); + IntervalTryAddition(result.months, number, MONTHS_PER_CENTURY, fraction); break; case DatePartSpecifier::DECADE: - IntervalTryAddition(result.months, number, MONTHS_PER_DECADE); + IntervalTryAddition(result.months, number, MONTHS_PER_DECADE, fraction); break; case DatePartSpecifier::YEAR: - IntervalTryAddition(result.months, number, MONTHS_PER_YEAR); + IntervalTryAddition(result.months, number, MONTHS_PER_YEAR, fraction); break; case DatePartSpecifier::QUARTER: - IntervalTryAddition(result.months, number, MONTHS_PER_QUARTER); + IntervalTryAddition(result.months, number, MONTHS_PER_QUARTER, fraction); + // Reduce to fraction of a month + fraction *= MONTHS_PER_QUARTER; + fraction %= MICROS_PER_SEC; + IntervalTryAddition(result.days, 0, DAYS_PER_MONTH, fraction); break; case DatePartSpecifier::MONTH: IntervalTryAddition(result.months, number, 1); + IntervalTryAddition(result.days, 0, DAYS_PER_MONTH, fraction); break; case DatePartSpecifier::DAY: IntervalTryAddition(result.days, number, 1); + IntervalTryAddition(result.micros, 0, MICROS_PER_DAY, fraction); break; case DatePartSpecifier::WEEK: - IntervalTryAddition(result.days, number, DAYS_PER_WEEK); + IntervalTryAddition(result.days, number, DAYS_PER_WEEK, fraction); + // Reduce to fraction of a day + fraction *= DAYS_PER_WEEK; + fraction %= MICROS_PER_SEC; + IntervalTryAddition(result.micros, 0, MICROS_PER_DAY, fraction); break; case DatePartSpecifier::MICROSECONDS: + // Round the fraction + number += (fraction * 2) / MICROS_PER_SEC; IntervalTryAddition(result.micros, number, 1); break; case DatePartSpecifier::MILLISECONDS: - IntervalTryAddition(result.micros, number, MICROS_PER_MSEC); + IntervalTryAddition(result.micros, number, MICROS_PER_MSEC, fraction); break; case DatePartSpecifier::SECOND: - IntervalTryAddition(result.micros, number, MICROS_PER_SEC); + IntervalTryAddition(result.micros, number, MICROS_PER_SEC, fraction); break; case DatePartSpecifier::MINUTE: - IntervalTryAddition(result.micros, number, MICROS_PER_MINUTE); + IntervalTryAddition(result.micros, number, MICROS_PER_MINUTE, fraction); break; case DatePartSpecifier::HOUR: - IntervalTryAddition(result.micros, number, MICROS_PER_HOUR); + IntervalTryAddition(result.micros, number, MICROS_PER_HOUR, fraction); break; default: HandleCastError::AssignError( @@ -492,6 +513,10 @@ dtime_t Interval::Add(dtime_t left, interval_t right, date_t &date) { return left; } +dtime_tz_t Interval::Add(dtime_tz_t left, interval_t right, date_t &date) { + return dtime_tz_t(Interval::Add(left.time(), right, date), left.offset()); +} + timestamp_t Interval::Add(timestamp_t left, interval_t right) { if (!Timestamp::IsFinite(left)) { return left; diff --git a/src/duckdb/src/common/types/list_segment.cpp b/src/duckdb/src/common/types/list_segment.cpp index 6627383af..f0308268b 100644 --- a/src/duckdb/src/common/types/list_segment.cpp +++ b/src/duckdb/src/common/types/list_segment.cpp @@ -1,4 +1,5 @@ #include "duckdb/common/types/list_segment.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -338,10 +339,7 @@ static void WriteDataToArraySegment(const ListSegmentFunctions &functions, Arena auto valid = input_data.unified.validity.RowIsValid(sel_entry_idx); null_mask[segment->count] = !valid; - if (!valid) { - return; - } - + // Arrays require there to be values in the child even when the entry is NULL. auto array_size = ArrayType::GetSize(input_data.logical_type); auto array_offset = sel_entry_idx * array_size; @@ -523,10 +521,8 @@ static void ReadDataFromArraySegment(const ListSegmentFunctions &functions, cons 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 { +void ListSegmentFunctions::BuildListVector(const LinkedList &linked_list, Vector &result, idx_t total_count) const { auto &read_data_from_segment = *this; - idx_t total_count = initial_total_count; auto segment = linked_list.first_segment; while (segment) { read_data_from_segment.read_data(read_data_from_segment, segment, result, total_count); @@ -590,6 +586,9 @@ void GetSegmentDataFunctions(ListSegmentFunctions &functions, const LogicalType case PhysicalType::INT128: SegmentPrimitiveFunction(functions); break; + case PhysicalType::UINT128: + SegmentPrimitiveFunction(functions); + break; case PhysicalType::INTERVAL: SegmentPrimitiveFunction(functions); break; diff --git a/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp b/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp index 03fded4f7..e4ba93797 100644 --- a/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp +++ b/src/duckdb/src/common/types/row/partitioned_tuple_data.cpp @@ -364,7 +364,7 @@ void PartitionedTupleData::Unpin() { } } -vector> &PartitionedTupleData::GetPartitions() { +unsafe_vector> &PartitionedTupleData::GetPartitions() { return partitions; } @@ -400,6 +400,16 @@ idx_t PartitionedTupleData::PartitionCount() const { return partitions.size(); } +void PartitionedTupleData::GetSizesAndCounts(vector &partition_sizes, vector &partition_counts) const { + D_ASSERT(partition_sizes.size() == PartitionCount()); + D_ASSERT(partition_sizes.size() == partition_counts.size()); + for (idx_t i = 0; i < PartitionCount(); i++) { + auto &partition = *partitions[i]; + partition_sizes[i] += partition.SizeInBytes(); + partition_counts[i] += partition.Count(); + } +} + void PartitionedTupleData::Verify() const { #ifdef DEBUG idx_t total_count = 0; 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 aa4ceec03..06bccdb01 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 @@ -2,6 +2,7 @@ #include "duckdb/common/fast_mem.hpp" #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/row/tuple_data_collection.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -927,6 +928,9 @@ TupleDataScatterFunction TupleDataCollection::GetScatterFunction(const LogicalTy case PhysicalType::UINT64: result.function = TupleDataGetScatterFunction(within_collection); break; + case PhysicalType::UINT128: + result.function = TupleDataGetScatterFunction(within_collection); + break; case PhysicalType::FLOAT: result.function = TupleDataGetScatterFunction(within_collection); break; @@ -1506,6 +1510,9 @@ TupleDataGatherFunction TupleDataCollection::GetGatherFunction(const LogicalType case PhysicalType::UINT64: result.function = TupleDataGetGatherFunction(within_collection); break; + case PhysicalType::UINT128: + result.function = TupleDataGetGatherFunction(within_collection); + break; case PhysicalType::FLOAT: result.function = TupleDataGetGatherFunction(within_collection); break; diff --git a/src/duckdb/src/common/types/time.cpp b/src/duckdb/src/common/types/time.cpp index cd6e84bda..8ef18c2f2 100644 --- a/src/duckdb/src/common/types/time.cpp +++ b/src/duckdb/src/common/types/time.cpp @@ -7,6 +7,7 @@ #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/operator/multiply.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" #include #include @@ -132,71 +133,6 @@ bool Time::TryConvertTime(const char *buf, idx_t len, idx_t &pos, dtime_t &resul return result.micros <= Interval::MICROS_PER_DAY; } -bool Time::TryParseUTCOffset(const char *str, idx_t &pos, idx_t len, int32_t &offset) { - offset = 0; - if (pos == len || StringUtil::CharacterIsSpace(str[pos])) { - return true; - } - - idx_t curpos = pos; - // Minimum of 3 characters - if (curpos + 3 > len) { - // no characters left to parse - return false; - } - - const auto sign_char = str[curpos]; - if (sign_char != '+' && sign_char != '-') { - // expected either + or - - return false; - } - curpos++; - - int32_t hh = 0; - idx_t start = curpos; - for (; curpos < len; ++curpos) { - const auto c = str[curpos]; - if (!StringUtil::CharacterIsDigit(c)) { - break; - } - hh = hh * 10 + (c - '0'); - } - // HH is in [-1559,+1559] and must be at least two digits - if (curpos - start < 2 || hh > 1559) { - return false; - } - - // optional minute specifier: expected ":MM" - int32_t mm = 0; - if (curpos + 3 <= len && str[curpos] == ':') { - ++curpos; - if (!Date::ParseDoubleDigit(str, len, curpos, mm) || mm >= Interval::MINS_PER_HOUR) { - return false; - } - } - - // optional seconds specifier: expected ":SS" - int32_t ss = 0; - if (curpos + 3 <= len && str[curpos] == ':') { - ++curpos; - if (!Date::ParseDoubleDigit(str, len, curpos, ss) || ss >= Interval::SECS_PER_MINUTE) { - return false; - } - } - - // Assemble the offset now that we know nothing went wrong - offset += hh * Interval::SECS_PER_HOUR; - offset += mm * Interval::SECS_PER_MINUTE; - offset += ss; - if (sign_char == '-') { - offset = -offset; - } - - pos = curpos; - - return true; -} - bool Time::TryConvertTimeTZ(const char *buf, idx_t len, idx_t &pos, dtime_tz_t &result, bool strict) { dtime_t time_part; if (!Time::TryConvertInternal(buf, len, pos, time_part, false)) { @@ -214,9 +150,33 @@ bool Time::TryConvertTimeTZ(const char *buf, idx_t len, idx_t &pos, dtime_tz_t & return false; } - // We can't use Timestamp::TryParseUTCOffset because the colon is optional there but required here. - int32_t offset = 0; - if (!TryParseUTCOffset(buf, pos, len, offset)) { + // skip optional whitespace before offset + while (pos < len && StringUtil::CharacterIsSpace(buf[pos])) { + pos++; + } + + // Get the ±HH[:MM] part + int hh = 0; + int mm = 0; + if (pos < len && !Timestamp::TryParseUTCOffset(buf, pos, len, hh, mm)) { + return false; + } + + // Offsets are in seconds in the open interval (-16:00:00, +16:00:00) + int32_t offset = ((hh * Interval::MINS_PER_HOUR) + mm) * Interval::SECS_PER_MINUTE; + + // Check for trailing seconds. + // (PG claims they don't support this but they do...) + if (pos < len && buf[pos] == ':') { + ++pos; + int ss = 0; + if (!Date::ParseDoubleDigit(buf, len, pos, ss)) { + return false; + } + offset += (offset < 0) ? -ss : ss; + } + + if (offset < dtime_tz_t::MIN_OFFSET || offset > dtime_tz_t::MAX_OFFSET) { return false; } @@ -237,6 +197,11 @@ bool Time::TryConvertTimeTZ(const char *buf, idx_t len, idx_t &pos, dtime_tz_t & return true; } +dtime_t Time::NormalizeTimeTZ(dtime_tz_t timetz) { + date_t date(0); + return Interval::Add(timetz.time(), {0, 0, -timetz.offset() * Interval::MICROS_PER_SEC}, date); +} + string Time::ConversionError(const string &str) { return StringUtil::Format("time field value out of range: \"%s\", " "expected format is ([YYYY-MM-DD ]HH:MM:SS[.MS])", diff --git a/src/duckdb/src/common/types/timestamp.cpp b/src/duckdb/src/common/types/timestamp.cpp index f0f5796d1..aeb8ef53a 100644 --- a/src/duckdb/src/common/types/timestamp.cpp +++ b/src/duckdb/src/common/types/timestamp.cpp @@ -9,6 +9,7 @@ #include "duckdb/common/operator/add.hpp" #include "duckdb/common/operator/multiply.hpp" #include "duckdb/common/operator/subtract.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" #include "duckdb/common/limits.hpp" #include @@ -271,10 +272,22 @@ bool Timestamp::TryFromDatetime(date_t date, dtime_t time, timestamp_t &result) return Timestamp::IsFinite(result); } +bool Timestamp::TryFromDatetime(date_t date, dtime_tz_t timetz, timestamp_t &result) { + if (!TryFromDatetime(date, timetz.time(), result)) { + return false; + } + // Offset is in seconds + const auto offset = int64_t(timetz.offset() * Interval::MICROS_PER_SEC); + if (!TryAddOperator::Operation(result.value, -offset, result.value)) { + return false; + } + return Timestamp::IsFinite(result); +} + timestamp_t Timestamp::FromDatetime(date_t date, dtime_t time) { timestamp_t result; if (!TryFromDatetime(date, time, result)) { - throw Exception("Overflow exception in date/time -> timestamp conversion"); + throw ConversionException("Overflow exception in date/time -> timestamp conversion"); } return result; } @@ -342,9 +355,9 @@ int64_t Timestamp::GetEpochNanoSeconds(timestamp_t timestamp) { } double Timestamp::GetJulianDay(timestamp_t timestamp) { - double result = Timestamp::GetTime(timestamp).micros; + double result = double(Timestamp::GetTime(timestamp).micros); result /= Interval::MICROS_PER_DAY; - result += Date::ExtractJulianDay(Timestamp::GetDate(timestamp)); + result += double(Date::ExtractJulianDay(Timestamp::GetDate(timestamp))); return result; } diff --git a/src/duckdb/src/common/types/uhugeint.cpp b/src/duckdb/src/common/types/uhugeint.cpp new file mode 100644 index 000000000..3d33d8cc7 --- /dev/null +++ b/src/duckdb/src/common/types/uhugeint.cpp @@ -0,0 +1,746 @@ +#include "duckdb/common/types/uhugeint.hpp" +#include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/exception.hpp" +#include "duckdb/common/algorithm.hpp" +#include "duckdb/common/limits.hpp" +#include "duckdb/common/windows_undefs.hpp" +#include "duckdb/common/types/value.hpp" +#include "duckdb/common/operator/cast_operators.hpp" + +#include +#include + +namespace duckdb { + +//===--------------------------------------------------------------------===// +// String Conversion +//===--------------------------------------------------------------------===// +const uhugeint_t Uhugeint::POWERS_OF_TEN[] { + uhugeint_t(1), + uhugeint_t(10), + uhugeint_t(100), + uhugeint_t(1000), + uhugeint_t(10000), + uhugeint_t(100000), + uhugeint_t(1000000), + uhugeint_t(10000000), + uhugeint_t(100000000), + uhugeint_t(1000000000), + uhugeint_t(10000000000), + uhugeint_t(100000000000), + uhugeint_t(1000000000000), + uhugeint_t(10000000000000), + uhugeint_t(100000000000000), + uhugeint_t(1000000000000000), + uhugeint_t(10000000000000000), + uhugeint_t(100000000000000000), + uhugeint_t(1000000000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(10), + uhugeint_t(1000000000000000000) * uhugeint_t(100), + uhugeint_t(1000000000000000000) * uhugeint_t(1000), + uhugeint_t(1000000000000000000) * uhugeint_t(10000), + uhugeint_t(1000000000000000000) * uhugeint_t(100000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000), + uhugeint_t(1000000000000000000) * uhugeint_t(10000000), + uhugeint_t(1000000000000000000) * uhugeint_t(100000000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(10000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(100000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(10000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(100000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(10000000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(100000000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000000000000), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000000000000) * uhugeint_t(10), + uhugeint_t(1000000000000000000) * uhugeint_t(1000000000000000000) * uhugeint_t(100)}; + +string Uhugeint::ToString(uhugeint_t input) { + uhugeint_t remainder; + string result; + while (true) { + if (!input.lower && !input.upper) { + break; + } + input = Uhugeint::DivMod(input, 10, remainder); + result = string(1, '0' + remainder.lower) + result; // NOLINT + } + if (result.empty()) { + // value is zero + return "0"; + } + return result; +} + +//===--------------------------------------------------------------------===// +// Negate +//===--------------------------------------------------------------------===// + +template <> +void Uhugeint::NegateInPlace(uhugeint_t &input) { + uhugeint_t result = 0; + result -= input; + input = result; +} + +bool Uhugeint::TryNegate(uhugeint_t input, uhugeint_t &result) { + // unsigned integers can always be negated + Uhugeint::NegateInPlace(input); + result = input; + return true; +} + +//===--------------------------------------------------------------------===// +// Multiply +//===--------------------------------------------------------------------===// +bool Uhugeint::TryMultiply(uhugeint_t lhs, uhugeint_t rhs, uhugeint_t &result) { +#if ((__GNUC__ >= 5) || defined(__clang__)) && defined(__SIZEOF_INT128__) + __uint128_t left = __uint128_t(lhs.lower) + (__uint128_t(lhs.upper) << 64); + __uint128_t right = __uint128_t(rhs.lower) + (__uint128_t(rhs.upper) << 64); + __uint128_t result_u128; + if (__builtin_mul_overflow(left, right, &result_u128)) { + return false; + } + result.upper = uint64_t(result_u128 >> 64); + result.lower = uint64_t(result_u128 & 0xffffffffffffffff); +#else + // split values into 4 32-bit parts + uint64_t top[4] = {lhs.upper >> 32, lhs.upper & 0xffffffff, lhs.lower >> 32, lhs.lower & 0xffffffff}; + uint64_t bottom[4] = {rhs.upper >> 32, rhs.upper & 0xffffffff, rhs.lower >> 32, rhs.lower & 0xffffffff}; + uint64_t products[4][4]; + + // multiply each component of the values + for (int y = 3; y > -1; y--) { + for (int x = 3; x > -1; x--) { + products[3 - x][y] = top[x] * bottom[y]; + } + } + + // if any of these products are set to a non-zero value, there is always an overflow + if (products[2][1] || products[1][0] || products[2][0]) { + return false; + } + + // if the high bits of any of these are set, there is always an overflow + if (products[1][1] & 0xffffffff00000000 || products[3][0] & 0xffffffff00000000 || + products[3][3] & 0xffffffff00000000 || products[3][2] & 0xffffffff00000000 || + products[3][1] & 0xffffffff00000000 || products[2][2] & 0xffffffff00000000 || + products[0][0] & 0xffffffff00000000) { + return false; + } + + // first row + uint64_t fourth32 = (products[0][3] & 0xffffffff); + uint64_t third32 = (products[0][2] & 0xffffffff) + (products[0][3] >> 32); + uint64_t second32 = (products[0][1] & 0xffffffff) + (products[0][2] >> 32); + uint64_t first32 = (products[0][0] & 0xffffffff) + (products[0][1] >> 32); + + // second row + third32 += (products[1][3] & 0xffffffff); + second32 += (products[1][2] & 0xffffffff) + (products[1][3] >> 32); + first32 += (products[1][1] & 0xffffffff) + (products[1][2] >> 32); + + // third row + second32 += (products[2][3] & 0xffffffff); + first32 += (products[2][2] & 0xffffffff) + (products[2][3] >> 32); + + // fourth row + first32 += (products[3][3] & 0xffffffff); + + // move carry to next digit + third32 += fourth32 >> 32; + second32 += third32 >> 32; + first32 += second32 >> 32; + + // remove carry from current digit + fourth32 &= 0xffffffff; + third32 &= 0xffffffff; + second32 &= 0xffffffff; + first32 &= 0xffffffff; + + // combine components + result.lower = (third32 << 32) | fourth32; + result.upper = (first32 << 32) | second32; +#endif + return true; +} + +// No overflow check, will wrap +template <> +uhugeint_t Uhugeint::Multiply(uhugeint_t lhs, uhugeint_t rhs) { + uhugeint_t result; +#if ((__GNUC__ >= 5) || defined(__clang__)) && defined(__SIZEOF_INT128__) + __uint128_t left = __uint128_t(lhs.lower) + (__uint128_t(lhs.upper) << 64); + __uint128_t right = __uint128_t(rhs.lower) + (__uint128_t(rhs.upper) << 64); + __uint128_t result_u128; + + result_u128 = left * right; + result.upper = uint64_t(result_u128 >> 64); + result.lower = uint64_t(result_u128 & 0xffffffffffffffff); +#else + // split values into 4 32-bit parts + uint64_t top[4] = {lhs.upper >> 32, lhs.upper & 0xffffffff, lhs.lower >> 32, lhs.lower & 0xffffffff}; + uint64_t bottom[4] = {rhs.upper >> 32, rhs.upper & 0xffffffff, rhs.lower >> 32, rhs.lower & 0xffffffff}; + uint64_t products[4][4]; + + // multiply each component of the values + for (int y = 3; y > -1; y--) { + for (int x = 3; x > -1; x--) { + products[3 - x][y] = top[x] * bottom[y]; + } + } + + // first row + uint64_t fourth32 = (products[0][3] & 0xffffffff); + uint64_t third32 = (products[0][2] & 0xffffffff) + (products[0][3] >> 32); + uint64_t second32 = (products[0][1] & 0xffffffff) + (products[0][2] >> 32); + uint64_t first32 = (products[0][0] & 0xffffffff) + (products[0][1] >> 32); + + // second row + third32 += (products[1][3] & 0xffffffff); + second32 += (products[1][2] & 0xffffffff) + (products[1][3] >> 32); + first32 += (products[1][1] & 0xffffffff) + (products[1][2] >> 32); + + // third row + second32 += (products[2][3] & 0xffffffff); + first32 += (products[2][2] & 0xffffffff) + (products[2][3] >> 32); + + // fourth row + first32 += (products[3][3] & 0xffffffff); + + // move carry to next digit + third32 += fourth32 >> 32; + second32 += third32 >> 32; + first32 += second32 >> 32; + + // remove carry from current digit + fourth32 &= 0xffffffff; + third32 &= 0xffffffff; + second32 &= 0xffffffff; + first32 &= 0xffffffff; + + // combine components + result.lower = (third32 << 32) | fourth32; + result.upper = (first32 << 32) | second32; +#endif + return result; +} + +//===--------------------------------------------------------------------===// +// Divide +//===--------------------------------------------------------------------===// + +int Sign(uhugeint_t n) { + return (n > 0); +} + +uhugeint_t Abs(uhugeint_t n) { + return (n); +} + +static uint8_t Bits(uhugeint_t x) { + uint8_t out = 0; + if (x.upper) { + out = 64; + for (uint64_t upper = x.upper; upper; upper >>= 1) { + ++out; + } + } else { + for (uint64_t lower = x.lower; lower; lower >>= 1) { + ++out; + } + } + return out; +} + +uhugeint_t Uhugeint::DivMod(uhugeint_t lhs, uhugeint_t rhs, uhugeint_t &remainder) { + if (rhs == 0) { + remainder = lhs; + return uhugeint_t(0); + } + + remainder = uhugeint_t(0); + if (rhs == uhugeint_t(1)) { + return lhs; + } else if (lhs == rhs) { + return uhugeint_t(1); + } else if (lhs == uhugeint_t(0) || lhs < rhs) { + remainder = lhs; + return uhugeint_t(0); + } + + uhugeint_t result = 0; + for (uint8_t idx = Bits(lhs); idx > 0; --idx) { + result <<= 1; + remainder <<= 1; + + if (((lhs >> (idx - 1U)) & 1) != 0) { + remainder += 1; + } + + if (remainder >= rhs) { + remainder -= rhs; + result += 1; + } + } + return result; +} + +template <> +uhugeint_t Uhugeint::Divide(uhugeint_t lhs, uhugeint_t rhs) { + uhugeint_t remainder; + return Uhugeint::DivMod(lhs, rhs, remainder); +} + +template <> +uhugeint_t Uhugeint::Modulo(uhugeint_t lhs, uhugeint_t rhs) { + uhugeint_t remainder; + (void)Uhugeint::DivMod(lhs, rhs, remainder); + return remainder; +} + +//===--------------------------------------------------------------------===// +// Add/Subtract +//===--------------------------------------------------------------------===// +bool Uhugeint::TryAddInPlace(uhugeint_t &lhs, uhugeint_t rhs) { + uint64_t new_upper = lhs.upper + rhs.upper; + bool no_overflow = !(new_upper < lhs.upper || new_upper < rhs.upper); + new_upper += (lhs.lower + rhs.lower) < lhs.lower; + if (new_upper < lhs.upper || new_upper < rhs.upper) { + no_overflow = false; + } + lhs.upper = new_upper; + lhs.lower += rhs.lower; + return no_overflow; +} + +bool Uhugeint::TrySubtractInPlace(uhugeint_t &lhs, uhugeint_t rhs) { + uint64_t new_upper = lhs.upper - rhs.upper - ((lhs.lower - rhs.lower) > lhs.lower); + bool no_overflow = !(new_upper > lhs.upper); + lhs.lower -= rhs.lower; + lhs.upper = new_upper; + return no_overflow; +} + +template <> +uhugeint_t Uhugeint::Add(uhugeint_t lhs, uhugeint_t rhs) { + return lhs + rhs; +} + +template <> +uhugeint_t Uhugeint::Subtract(uhugeint_t lhs, uhugeint_t rhs) { + return lhs - rhs; +} + +//===--------------------------------------------------------------------===// +// Cast/Conversion +//===--------------------------------------------------------------------===// +template +bool UhugeintTryCastInteger(uhugeint_t input, DST &result) { + if (input.upper == 0 && input.lower <= uint64_t(NumericLimits::Maximum())) { + result = DST(input.lower); + return true; + } + return false; +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, int8_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, int16_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, int32_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, int64_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, uint8_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, uint16_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, uint32_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, uint64_t &result) { + return UhugeintTryCastInteger(input, result); +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, uhugeint_t &result) { + result = input; + return true; +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, hugeint_t &result) { + if (input > uhugeint_t(NumericLimits::Maximum())) { + return false; + } + + result.lower = input.lower; + result.upper = input.upper; + return true; +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, float &result) { + double dbl_result; + Uhugeint::TryCast(input, dbl_result); + result = (float)dbl_result; + return true; +} + +template +bool CastUhugeintToFloating(uhugeint_t input, REAL_T &result) { + result = REAL_T(input.lower) + REAL_T(input.upper) * REAL_T(NumericLimits::Maximum()); + return true; +} + +template <> +bool Uhugeint::TryCast(uhugeint_t input, double &result) { + return CastUhugeintToFloating(input, result); +} + +template +uhugeint_t UhugeintConvertInteger(DST input) { + uhugeint_t result; + result.lower = (uint64_t)input; + result.upper = 0; + return result; +} + +template <> +bool Uhugeint::TryConvert(const char *value, uhugeint_t &result) { + auto len = strlen(value); + string_t string_val(value, len); + return TryCast::Operation(string_val, result, true); +} + +template <> +bool Uhugeint::TryConvert(int8_t value, uhugeint_t &result) { + if (value < 0) { + return false; + } + result = UhugeintConvertInteger(value); + return true; +} + +template <> +bool Uhugeint::TryConvert(int16_t value, uhugeint_t &result) { + if (value < 0) { + return false; + } + result = UhugeintConvertInteger(value); + return true; +} + +template <> +bool Uhugeint::TryConvert(int32_t value, uhugeint_t &result) { + if (value < 0) { + return false; + } + result = UhugeintConvertInteger(value); + return true; +} + +template <> +bool Uhugeint::TryConvert(int64_t value, uhugeint_t &result) { + if (value < 0) { + return false; + } + result = UhugeintConvertInteger(value); + return true; +} +template <> +bool Uhugeint::TryConvert(uint8_t value, uhugeint_t &result) { + result = UhugeintConvertInteger(value); + return true; +} +template <> +bool Uhugeint::TryConvert(uint16_t value, uhugeint_t &result) { + result = UhugeintConvertInteger(value); + return true; +} +template <> +bool Uhugeint::TryConvert(uint32_t value, uhugeint_t &result) { + result = UhugeintConvertInteger(value); + return true; +} +template <> +bool Uhugeint::TryConvert(uint64_t value, uhugeint_t &result) { + result = UhugeintConvertInteger(value); + return true; +} + +template <> +bool Uhugeint::TryConvert(uhugeint_t value, uhugeint_t &result) { + result = value; + return true; +} + +template <> +bool Uhugeint::TryConvert(float value, uhugeint_t &result) { + return Uhugeint::TryConvert(double(value), result); +} + +template +bool ConvertFloatingToUhugeint(REAL_T value, uhugeint_t &result) { + if (!Value::IsFinite(value)) { + return false; + } + if (value < 0 || value >= 340282366920938463463374607431768211456.0) { + return false; + } + result.lower = (uint64_t)fmod(value, REAL_T(NumericLimits::Maximum())); + result.upper = (uint64_t)(value / REAL_T(NumericLimits::Maximum())); + return true; +} + +template <> +bool Uhugeint::TryConvert(double value, uhugeint_t &result) { + return ConvertFloatingToUhugeint(value, result); +} + +template <> +bool Uhugeint::TryConvert(long double value, uhugeint_t &result) { + return ConvertFloatingToUhugeint(value, result); +} + +//===--------------------------------------------------------------------===// +// uhugeint_t operators +//===--------------------------------------------------------------------===// +uhugeint_t::uhugeint_t(uint64_t value) { + this->lower = value; + this->upper = 0; +} + +bool uhugeint_t::operator==(const uhugeint_t &rhs) const { + return Uhugeint::Equals(*this, rhs); +} + +bool uhugeint_t::operator!=(const uhugeint_t &rhs) const { + return Uhugeint::NotEquals(*this, rhs); +} + +bool uhugeint_t::operator<(const uhugeint_t &rhs) const { + return Uhugeint::LessThan(*this, rhs); +} + +bool uhugeint_t::operator<=(const uhugeint_t &rhs) const { + return Uhugeint::LessThanEquals(*this, rhs); +} + +bool uhugeint_t::operator>(const uhugeint_t &rhs) const { + return Uhugeint::GreaterThan(*this, rhs); +} + +bool uhugeint_t::operator>=(const uhugeint_t &rhs) const { + return Uhugeint::GreaterThanEquals(*this, rhs); +} + +uhugeint_t uhugeint_t::operator+(const uhugeint_t &rhs) const { + return uhugeint_t(upper + rhs.upper + ((lower + rhs.lower) < lower), lower + rhs.lower); +} + +uhugeint_t uhugeint_t::operator-(const uhugeint_t &rhs) const { + return uhugeint_t(upper - rhs.upper - ((lower - rhs.lower) > lower), lower - rhs.lower); +} + +uhugeint_t uhugeint_t::operator*(const uhugeint_t &rhs) const { + uhugeint_t result = *this; + result *= rhs; + return result; +} + +uhugeint_t uhugeint_t::operator/(const uhugeint_t &rhs) const { + return Uhugeint::Divide(*this, rhs); +} + +uhugeint_t uhugeint_t::operator%(const uhugeint_t &rhs) const { + return Uhugeint::Modulo(*this, rhs); +} + +uhugeint_t uhugeint_t::operator-() const { + return Uhugeint::Negate(*this); +} + +uhugeint_t uhugeint_t::operator>>(const uhugeint_t &rhs) const { + const uint64_t shift = rhs.lower; + if (rhs.upper != 0 || shift >= 128) { + return uhugeint_t(0); + } else if (shift == 0) { + return *this; + } else if (shift == 64) { + return uhugeint_t(0, upper); + } else if (shift < 64) { + return uhugeint_t(upper >> shift, (upper << (64 - shift)) + (lower >> shift)); + } else if ((128 > shift) && (shift > 64)) { + return uhugeint_t(0, (upper >> (shift - 64))); + } + return uhugeint_t(0); +} + +uhugeint_t uhugeint_t::operator<<(const uhugeint_t &rhs) const { + const uint64_t shift = rhs.lower; + if (rhs.upper != 0 || shift >= 128) { + return uhugeint_t(0); + } else if (shift == 0) { + return *this; + } else if (shift == 64) { + return uhugeint_t(lower, 0); + } else if (shift < 64) { + return uhugeint_t((upper << shift) + (lower >> (64 - shift)), lower << shift); + } else if ((128 > shift) && (shift > 64)) { + return uhugeint_t(lower << (shift - 64), 0); + } + return uhugeint_t(0); +} + +uhugeint_t uhugeint_t::operator&(const uhugeint_t &rhs) const { + uhugeint_t result; + result.lower = lower & rhs.lower; + result.upper = upper & rhs.upper; + return result; +} + +uhugeint_t uhugeint_t::operator|(const uhugeint_t &rhs) const { + uhugeint_t result; + result.lower = lower | rhs.lower; + result.upper = upper | rhs.upper; + return result; +} + +uhugeint_t uhugeint_t::operator^(const uhugeint_t &rhs) const { + uhugeint_t result; + result.lower = lower ^ rhs.lower; + result.upper = upper ^ rhs.upper; + return result; +} + +uhugeint_t uhugeint_t::operator~() const { + uhugeint_t result; + result.lower = ~lower; + result.upper = ~upper; + return result; +} + +uhugeint_t &uhugeint_t::operator+=(const uhugeint_t &rhs) { + *this = *this + rhs; + return *this; +} + +uhugeint_t &uhugeint_t::operator-=(const uhugeint_t &rhs) { + *this = *this - rhs; + return *this; +} + +uhugeint_t &uhugeint_t::operator*=(const uhugeint_t &rhs) { + *this = Uhugeint::Multiply(*this, rhs); + return *this; +} + +uhugeint_t &uhugeint_t::operator/=(const uhugeint_t &rhs) { + *this = Uhugeint::Divide(*this, rhs); + return *this; +} + +uhugeint_t &uhugeint_t::operator%=(const uhugeint_t &rhs) { + *this = Uhugeint::Modulo(*this, rhs); + return *this; +} + +uhugeint_t &uhugeint_t::operator>>=(const uhugeint_t &rhs) { + *this = *this >> rhs; + return *this; +} + +uhugeint_t &uhugeint_t::operator<<=(const uhugeint_t &rhs) { + *this = *this << rhs; + return *this; +} + +uhugeint_t &uhugeint_t::operator&=(const uhugeint_t &rhs) { + lower &= rhs.lower; + upper &= rhs.upper; + return *this; +} + +uhugeint_t &uhugeint_t::operator|=(const uhugeint_t &rhs) { + lower |= rhs.lower; + upper |= rhs.upper; + return *this; +} + +uhugeint_t &uhugeint_t::operator^=(const uhugeint_t &rhs) { + lower ^= rhs.lower; + upper ^= rhs.upper; + return *this; +} + +bool uhugeint_t::operator!() const { + return *this == 0; +} + +uhugeint_t::operator bool() const { + return *this != 0; +} + +template +static T NarrowCast(const uhugeint_t &input) { + // NarrowCast is supposed to truncate (take lower) + return static_cast(input.lower); +} + +uhugeint_t::operator uint8_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator uint16_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator uint32_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator uint64_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator int8_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator int16_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator int32_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator int64_t() const { + return NarrowCast(*this); +} +uhugeint_t::operator hugeint_t() const { + return {static_cast(this->upper), this->lower}; +} + +string uhugeint_t::ToString() const { + return Uhugeint::ToString(*this); +} + +} // namespace duckdb diff --git a/src/duckdb/src/common/types/value.cpp b/src/duckdb/src/common/types/value.cpp index 42cf680eb..377cfc93c 100644 --- a/src/duckdb/src/common/types/value.cpp +++ b/src/duckdb/src/common/types/value.cpp @@ -7,6 +7,7 @@ #include "duckdb/common/operator/cast_operators.hpp" #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/uhugeint.hpp" #include "utf8proc_wrapper.hpp" #include "duckdb/common/operator/numeric_binary_operators.hpp" #include "duckdb/common/printer.hpp" @@ -155,7 +156,7 @@ Value::Value(string_t val) : Value(val.GetString()) { Value::Value(string val) : type_(LogicalType::VARCHAR), is_null(false) { if (!Value::StringIsValid(val.c_str(), val.size())) { - throw Exception(ErrorManager::InvalidUnicodeError(val, "value construction")); + throw ErrorManager::InvalidUnicodeError(val, "value construction"); } value_info_ = make_shared(std::move(val)); } @@ -206,6 +207,8 @@ Value Value::MinimumValue(const LogicalType &type) { return Value::BIGINT(NumericLimits::Minimum()); case LogicalTypeId::HUGEINT: return Value::HUGEINT(NumericLimits::Minimum()); + case LogicalTypeId::UHUGEINT: + return Value::UHUGEINT(NumericLimits::Minimum()); case LogicalTypeId::UUID: return Value::UUID(NumericLimits::Minimum()); case LogicalTypeId::UTINYINT: @@ -230,7 +233,8 @@ Value Value::MinimumValue(const LogicalType &type) { case LogicalTypeId::TIMESTAMP_NS: return Value::TIMESTAMPNS(timestamp_t(NumericLimits::Minimum())); case LogicalTypeId::TIME_TZ: - return Value::TIMETZ(dtime_tz_t(dtime_t(0), dtime_tz_t::MIN_OFFSET)); + // "00:00:00+1559" from the PG docs, but actually 00:00:00+15:59:59 + return Value::TIMETZ(dtime_tz_t(dtime_t(0), dtime_tz_t::MAX_OFFSET)); case LogicalTypeId::TIMESTAMP_TZ: return Value::TIMESTAMPTZ(Timestamp::FromDatetime( Date::FromDate(Timestamp::MIN_YEAR, Timestamp::MIN_MONTH, Timestamp::MIN_DAY), dtime_t(0))); @@ -276,6 +280,8 @@ Value Value::MaximumValue(const LogicalType &type) { return Value::BIGINT(NumericLimits::Maximum()); case LogicalTypeId::HUGEINT: return Value::HUGEINT(NumericLimits::Maximum()); + case LogicalTypeId::UHUGEINT: + return Value::UHUGEINT(NumericLimits::Maximum()); case LogicalTypeId::UUID: return Value::UUID(NumericLimits::Maximum()); case LogicalTypeId::UTINYINT: @@ -300,8 +306,8 @@ Value Value::MaximumValue(const LogicalType &type) { case LogicalTypeId::TIMESTAMP_SEC: return MaximumValue(LogicalType::TIMESTAMP).DefaultCastAs(LogicalType::TIMESTAMP_S); case LogicalTypeId::TIME_TZ: - return Value::TIMETZ( - dtime_tz_t(dtime_t(Interval::SECS_PER_DAY * Interval::MICROS_PER_SEC - 1), dtime_tz_t::MAX_OFFSET)); + // "24:00:00-1559" from the PG docs but actually "24:00:00-15:59:59" + return Value::TIMETZ(dtime_tz_t(dtime_t(Interval::MICROS_PER_DAY), dtime_tz_t::MIN_OFFSET)); case LogicalTypeId::TIMESTAMP_TZ: return MaximumValue(LogicalType::TIMESTAMP); case LogicalTypeId::FLOAT: @@ -419,6 +425,13 @@ Value Value::HUGEINT(hugeint_t value) { return result; } +Value Value::UHUGEINT(uhugeint_t value) { + Value result(LogicalType::UHUGEINT); + result.value_.uhugeint = value; + result.is_null = false; + return result; +} + Value Value::UUID(hugeint_t value) { Value result(LogicalType::UUID); result.value_.hugeint = value; @@ -907,6 +920,11 @@ Value Value::CreateValue(hugeint_t value) { return Value::HUGEINT(value); } +template <> +Value Value::CreateValue(uhugeint_t value) { + return Value::UHUGEINT(value); +} + template <> Value Value::CreateValue(date_t value) { return Value::DATE(value); @@ -1004,6 +1022,8 @@ T Value::GetValueInternal() const { case LogicalTypeId::HUGEINT: case LogicalTypeId::UUID: return Cast::Operation(value_.hugeint); + case LogicalTypeId::UHUGEINT: + return Cast::Operation(value_.uhugeint); case LogicalTypeId::DATE: return Cast::Operation(value_.date); case LogicalTypeId::TIME: @@ -1108,6 +1128,10 @@ uint64_t Value::GetValue() const { return GetValueInternal(); } template <> +uhugeint_t Value::GetValue() const { + return GetValueInternal(); +} +template <> string Value::GetValue() const { return ToString(); } @@ -1177,6 +1201,8 @@ Value Value::Numeric(const LogicalType &type, int64_t value) { return Value::UBIGINT(value); case LogicalTypeId::HUGEINT: return Value::HUGEINT(value); + case LogicalTypeId::UHUGEINT: + return Value::UHUGEINT(value); case LogicalTypeId::DECIMAL: return Value::DECIMAL(value, DecimalType::GetWidth(type), DecimalType::GetScale(type)); case LogicalTypeId::FLOAT: @@ -1234,6 +1260,21 @@ Value Value::Numeric(const LogicalType &type, hugeint_t value) { } } +Value Value::Numeric(const LogicalType &type, uhugeint_t value) { +#ifdef DEBUG + // perform a throwing cast to verify that the type fits + Value::UHUGEINT(value).DefaultCastAs(type); +#endif + switch (type.id()) { + case LogicalTypeId::UHUGEINT: + return Value::UHUGEINT(value); + case LogicalTypeId::UBIGINT: + return Value::UBIGINT(Uhugeint::Cast(value)); + default: + return Value::Numeric(type, Uhugeint::Cast(value)); + } +} + //===--------------------------------------------------------------------===// // GetValueUnsafe //===--------------------------------------------------------------------===// @@ -1297,6 +1338,12 @@ uint64_t Value::GetValueUnsafe() const { return value_.ubigint; } +template <> +uhugeint_t Value::GetValueUnsafe() const { + D_ASSERT(type_.InternalType() == PhysicalType::UINT128); + return value_.uhugeint; +} + template <> string Value::GetValueUnsafe() const { return StringValue::Get(*this); @@ -1331,6 +1378,12 @@ dtime_t Value::GetValueUnsafe() const { return value_.time; } +template <> +dtime_tz_t Value::GetValueUnsafe() const { + D_ASSERT(type_.InternalType() == PhysicalType::INT64); + return value_.timetz; +} + template <> timestamp_t Value::GetValueUnsafe() const { D_ASSERT(type_.InternalType() == PhysicalType::INT64); @@ -1386,18 +1439,23 @@ string Value::ToSQLString() const { case LogicalTypeId::ENUM: return "'" + StringUtil::Replace(ToString(), "'", "''") + "'"; case LogicalTypeId::STRUCT: { - string ret = "{"; + bool is_unnamed = StructType::IsUnnamed(type_); + string ret = is_unnamed ? "(" : "{"; auto &child_types = StructType::GetChildTypes(type_); auto &struct_values = StructValue::GetChildren(*this); 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(); + if (is_unnamed) { + ret += child.ToSQLString(); + } else { + ret += "'" + name + "': " + child.ToSQLString(); + } if (i < struct_values.size() - 1) { ret += ", "; } } - ret += "}"; + ret += is_unnamed ? ")" : "}"; return ret; } case LogicalTypeId::FLOAT: @@ -1490,6 +1548,10 @@ uint64_t UBigIntValue::Get(const Value &value) { return value.GetValueUnsafe(); } +uhugeint_t UhugeIntValue::Get(const Value &value) { + return value.GetValueUnsafe(); +} + float FloatValue::Get(const Value &value) { return value.GetValueUnsafe(); } @@ -1590,6 +1652,8 @@ hugeint_t IntegralValue::Get(const Value &value) { return UIntegerValue::Get(value); case PhysicalType::UINT64: return UBigIntValue::Get(value); + case PhysicalType::UINT128: + return static_cast(UhugeIntValue::Get(value)); default: throw InternalException("Invalid internal type \"%s\" for IntegralValue::Get", value.type().ToString()); } @@ -1761,6 +1825,9 @@ void Value::Serialize(Serializer &serializer) const { case PhysicalType::INT128: serializer.WriteProperty(102, "value", value_.hugeint); break; + case PhysicalType::UINT128: + serializer.WriteProperty(102, "value", value_.uhugeint); + break; case PhysicalType::FLOAT: serializer.WriteProperty(102, "value", value_.float_); break; @@ -1840,6 +1907,9 @@ Value Value::Deserialize(Deserializer &deserializer) { case PhysicalType::INT64: new_value.value_.bigint = deserializer.ReadProperty(102, "value"); break; + case PhysicalType::UINT128: + new_value.value_.uhugeint = deserializer.ReadProperty(102, "value"); + break; case PhysicalType::INT128: new_value.value_.hugeint = deserializer.ReadProperty(102, "value"); break; diff --git a/src/duckdb/src/common/types/vector.cpp b/src/duckdb/src/common/types/vector.cpp index eb2494f68..f9db436e0 100644 --- a/src/duckdb/src/common/types/vector.cpp +++ b/src/duckdb/src/common/types/vector.cpp @@ -9,6 +9,7 @@ #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/sel_cache.hpp" #include "duckdb/common/types/vector_cache.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/storage/buffer/buffer_handle.hpp" #include "duckdb/function/scalar/nested_functions.hpp" @@ -127,7 +128,7 @@ void Vector::ResetFromCache(const VectorCache &cache) { cache.ResetFromCache(*this); } -void Vector::Slice(Vector &other, idx_t offset, idx_t end) { +void Vector::Slice(const Vector &other, idx_t offset, idx_t end) { if (other.GetVectorType() == VectorType::CONSTANT_VECTOR) { Reference(other); return; @@ -162,7 +163,7 @@ void Vector::Slice(Vector &other, idx_t offset, idx_t end) { } } -void Vector::Slice(Vector &other, const SelectionVector &sel, idx_t count) { +void Vector::Slice(const Vector &other, const SelectionVector &sel, idx_t count) { Reference(other); Slice(sel, count); } @@ -406,6 +407,9 @@ void Vector::SetValue(idx_t index, const Value &val) { case PhysicalType::UINT64: reinterpret_cast(data)[index] = val.GetValueUnsafe(); break; + case PhysicalType::UINT128: + reinterpret_cast(data)[index] = val.GetValueUnsafe(); + break; case PhysicalType::FLOAT: reinterpret_cast(data)[index] = val.GetValueUnsafe(); break; @@ -560,6 +564,8 @@ Value Vector::GetValueInternal(const Vector &v_p, idx_t index_p) { return Value::TIMESTAMPTZ(reinterpret_cast(data)[index]); case LogicalTypeId::HUGEINT: return Value::HUGEINT(reinterpret_cast(data)[index]); + case LogicalTypeId::UHUGEINT: + return Value::UHUGEINT(reinterpret_cast(data)[index]); case LogicalTypeId::UUID: return Value::UUID(reinterpret_cast(data)[index]); case LogicalTypeId::DECIMAL: { @@ -848,6 +854,9 @@ void Vector::Flatten(idx_t count) { case PhysicalType::INT128: TemplatedFlattenConstantVector(data, old_data, count); break; + case PhysicalType::UINT128: + TemplatedFlattenConstantVector(data, old_data, count); + break; case PhysicalType::FLOAT: TemplatedFlattenConstantVector(data, old_data, count); break; @@ -2002,6 +2011,7 @@ void ListVector::SetListSize(Vector &vec, idx_t size) { if (vec.GetVectorType() == VectorType::DICTIONARY_VECTOR) { auto &child = DictionaryVector::Child(vec); ListVector::SetListSize(child, size); + return; } vec.auxiliary->Cast().SetSize(size); } diff --git a/src/duckdb/src/common/types/vector_buffer.cpp b/src/duckdb/src/common/types/vector_buffer.cpp index 1d999abf1..7a7db5fae 100644 --- a/src/duckdb/src/common/types/vector_buffer.cpp +++ b/src/duckdb/src/common/types/vector_buffer.cpp @@ -1,7 +1,6 @@ #include "duckdb/common/types/vector_buffer.hpp" #include "duckdb/common/assert.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/storage/buffer/buffer_handle.hpp" diff --git a/src/duckdb/src/common/value_operations/comparison_operations.cpp b/src/duckdb/src/common/value_operations/comparison_operations.cpp index 5ba5824e2..b2d59b0b5 100644 --- a/src/duckdb/src/common/value_operations/comparison_operations.cpp +++ b/src/duckdb/src/common/value_operations/comparison_operations.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/value_operations/value_operations.hpp" #include "duckdb/planner/expression/bound_comparison_expression.hpp" @@ -102,7 +103,7 @@ static bool TemplatedBooleanOperation(const Value &left, const Value &right) { Value left_copy = left; Value right_copy = right; - LogicalType comparison_type = BoundComparisonExpression::BindComparison(left_type, right_type); + auto comparison_type = LogicalType::ForceMaxLogicalType(left_type, right_type); if (!left_copy.DefaultTryCastAs(comparison_type) || !right_copy.DefaultTryCastAs(comparison_type)) { return false; } @@ -128,6 +129,8 @@ static bool TemplatedBooleanOperation(const Value &left, const Value &right) { return OP::Operation(left.GetValueUnsafe(), right.GetValueUnsafe()); case PhysicalType::UINT64: return OP::Operation(left.GetValueUnsafe(), right.GetValueUnsafe()); + case PhysicalType::UINT128: + return OP::Operation(left.GetValueUnsafe(), right.GetValueUnsafe()); case PhysicalType::INT128: return OP::Operation(left.GetValueUnsafe(), right.GetValueUnsafe()); case PhysicalType::FLOAT: diff --git a/src/duckdb/src/common/vector_operations/comparison_operators.cpp b/src/duckdb/src/common/vector_operations/comparison_operators.cpp index 6c97eee68..d73ed852b 100644 --- a/src/duckdb/src/common/vector_operations/comparison_operators.cpp +++ b/src/duckdb/src/common/vector_operations/comparison_operators.cpp @@ -6,6 +6,7 @@ #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/binary_executor.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" @@ -237,6 +238,9 @@ struct ComparisonExecutor { case PhysicalType::INT128: TemplatedExecute(left, right, result, count); break; + case PhysicalType::UINT128: + TemplatedExecute(left, right, result, count); + break; case PhysicalType::FLOAT: TemplatedExecute(left, right, result, count); break; diff --git a/src/duckdb/src/common/vector_operations/generators.cpp b/src/duckdb/src/common/vector_operations/generators.cpp index 77dd5f9ce..0b5e8f56a 100644 --- a/src/duckdb/src/common/vector_operations/generators.cpp +++ b/src/duckdb/src/common/vector_operations/generators.cpp @@ -13,7 +13,7 @@ template void TemplatedGenerateSequence(Vector &result, idx_t count, int64_t start, int64_t increment) { D_ASSERT(result.GetType().IsNumeric()); if (start > NumericLimits::Maximum() || increment > NumericLimits::Maximum()) { - throw Exception("Sequence start or increment out of type range"); + throw InternalException("Sequence start or increment out of type range"); } result.SetVectorType(VectorType::FLAT_VECTOR); auto result_data = FlatVector::GetData(result); @@ -59,7 +59,7 @@ void TemplatedGenerateSequence(Vector &result, idx_t count, const SelectionVecto int64_t increment) { D_ASSERT(result.GetType().IsNumeric()); if (start > NumericLimits::Maximum() || increment > NumericLimits::Maximum()) { - throw Exception("Sequence start or increment out of type range"); + throw InternalException("Sequence start or increment out of type range"); } result.SetVectorType(VectorType::FLAT_VECTOR); auto result_data = FlatVector::GetData(result); 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 c14ff0c3a..fc5d98a85 100644 --- a/src/duckdb/src/common/vector_operations/is_distinct_from.cpp +++ b/src/duckdb/src/common/vector_operations/is_distinct_from.cpp @@ -1,3 +1,4 @@ +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/operator/comparison_operators.hpp" @@ -879,6 +880,9 @@ static void ExecuteDistinct(Vector &left, Vector &right, Vector &result, idx_t c case PhysicalType::INT128: TemplatedDistinctExecute(left, right, result, count); break; + case PhysicalType::UINT128: + TemplatedDistinctExecute(left, right, result, count); + break; case PhysicalType::FLOAT: TemplatedDistinctExecute(left, right, result, count); break; @@ -925,6 +929,8 @@ static idx_t TemplatedDistinctSelectOperation(Vector &left, Vector &right, const return DistinctSelect(left, right, sel, count, true_sel, false_sel); case PhysicalType::INT128: return DistinctSelect(left, right, sel, count, true_sel, false_sel); + case PhysicalType::UINT128: + return DistinctSelect(left, right, sel, count, true_sel, false_sel); case PhysicalType::FLOAT: return DistinctSelect(left, right, sel, count, true_sel, false_sel); case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/common/vector_operations/vector_copy.cpp b/src/duckdb/src/common/vector_operations/vector_copy.cpp index 52016757f..7921743ae 100644 --- a/src/duckdb/src/common/vector_operations/vector_copy.cpp +++ b/src/duckdb/src/common/vector_operations/vector_copy.cpp @@ -6,7 +6,7 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/types/null_value.hpp" -#include "duckdb/common/types/chunk_collection.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/storage/segment/uncompressed.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" @@ -102,7 +102,7 @@ void VectorOperations::Copy(const Vector &source_p, Vector &target, const Select } } else { auto &smask = CopyValidityMask(*source); - if (smask.IsMaskSet()) { + if (smask.IsMaskSet() || tmask.IsMaskSet()) { for (idx_t i = 0; i < copy_count; i++) { auto idx = sel->get_index(source_offset + i); @@ -160,6 +160,9 @@ void VectorOperations::Copy(const Vector &source_p, Vector &target, const Select case PhysicalType::INT128: TemplatedCopy(*source, *sel, target, source_offset, target_offset, copy_count); break; + case PhysicalType::UINT128: + TemplatedCopy(*source, *sel, target, source_offset, target_offset, copy_count); + break; case PhysicalType::FLOAT: TemplatedCopy(*source, *sel, target, source_offset, target_offset, copy_count); break; diff --git a/src/duckdb/src/common/vector_operations/vector_hash.cpp b/src/duckdb/src/common/vector_operations/vector_hash.cpp index 422529a38..84d6c1783 100644 --- a/src/duckdb/src/common/vector_operations/vector_hash.cpp +++ b/src/duckdb/src/common/vector_operations/vector_hash.cpp @@ -3,6 +3,7 @@ // Description: This file contains the vectorized hash implementations //===--------------------------------------------------------------------===// +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/types/hash.hpp" @@ -237,6 +238,9 @@ static inline void HashTypeSwitch(Vector &input, Vector &result, const Selection case PhysicalType::INT128: TemplatedLoopHash(input, result, rsel, count); break; + case PhysicalType::UINT128: + TemplatedLoopHash(input, result, rsel, count); + break; case PhysicalType::FLOAT: TemplatedLoopHash(input, result, rsel, count); break; @@ -373,6 +377,9 @@ static inline void CombineHashTypeSwitch(Vector &hashes, Vector &input, const Se case PhysicalType::INT128: TemplatedLoopCombineHash(input, hashes, rsel, count); break; + case PhysicalType::UINT128: + TemplatedLoopCombineHash(input, hashes, rsel, count); + break; case PhysicalType::FLOAT: TemplatedLoopCombineHash(input, hashes, rsel, count); break; diff --git a/src/duckdb/src/common/vector_operations/vector_storage.cpp b/src/duckdb/src/common/vector_operations/vector_storage.cpp index be7c97c71..9c399519d 100644 --- a/src/duckdb/src/common/vector_operations/vector_storage.cpp +++ b/src/duckdb/src/common/vector_operations/vector_storage.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/types/null_value.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" namespace duckdb { @@ -54,6 +55,9 @@ void VectorOperations::WriteToStorage(Vector &source, idx_t count, data_ptr_t ta case PhysicalType::INT128: CopyToStorageLoop(vdata, count, target); break; + case PhysicalType::UINT128: + CopyToStorageLoop(vdata, count, target); + break; case PhysicalType::FLOAT: CopyToStorageLoop(vdata, count, target); break; @@ -108,6 +112,9 @@ void VectorOperations::ReadFromStorage(data_ptr_t source, idx_t count, Vector &r case PhysicalType::INT128: ReadFromStorageLoop(source, count, result); break; + case PhysicalType::UINT128: + ReadFromStorageLoop(source, count, result); + break; case PhysicalType::FLOAT: ReadFromStorageLoop(source, count, result); break; diff --git a/src/duckdb/src/common/virtual_file_system.cpp b/src/duckdb/src/common/virtual_file_system.cpp index 0aaff1423..3337623e4 100644 --- a/src/duckdb/src/common/virtual_file_system.cpp +++ b/src/duckdb/src/common/virtual_file_system.cpp @@ -1,5 +1,4 @@ #include "duckdb/common/virtual_file_system.hpp" - #include "duckdb/common/gzip_file_system.hpp" #include "duckdb/common/pipe_file_system.hpp" #include "duckdb/common/string_util.hpp" diff --git a/src/duckdb/src/core_functions/aggregate/distributive/approx_count.cpp b/src/duckdb/src/core_functions/aggregate/distributive/approx_count.cpp index 599ecb7d1..844e31acf 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/approx_count.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/approx_count.cpp @@ -129,16 +129,17 @@ AggregateFunctionSet ApproxCountDistinctFun::GetFunctions() { approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::USMALLINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::UINTEGER)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::UBIGINT)); + approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::UHUGEINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::TINYINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::SMALLINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::BIGINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::HUGEINT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::FLOAT)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::DOUBLE)); - approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::VARCHAR)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::TIMESTAMP)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::TIMESTAMP_TZ)); approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::BLOB)); + approx_count.AddFunction(GetApproxCountDistinctFunction(LogicalType::ANY_PARAMS(LogicalType::VARCHAR, 150))); return approx_count; } diff --git a/src/duckdb/src/core_functions/aggregate/distributive/arg_min_max.cpp b/src/duckdb/src/core_functions/aggregate/distributive/arg_min_max.cpp index ef1c714c5..b6d69457a 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/arg_min_max.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/arg_min_max.cpp @@ -10,7 +10,7 @@ namespace duckdb { struct ArgMinMaxStateBase { - ArgMinMaxStateBase() : is_initialized(false) { + ArgMinMaxStateBase() : is_initialized(false), arg_null(false) { } template @@ -22,7 +22,7 @@ struct ArgMinMaxStateBase { } template - static inline void AssignValue(T &target, T new_value, bool is_initialized) { + static inline void AssignValue(T &target, T new_value) { target = new_value; } @@ -32,9 +32,15 @@ struct ArgMinMaxStateBase { } bool is_initialized; + bool arg_null; }; // Out-of-line specialisations +template <> +void ArgMinMaxStateBase::CreateValue(string_t &value) { + value = string_t(uint32_t(0)); +} + template <> void ArgMinMaxStateBase::CreateValue(Vector *&value) { value = nullptr; @@ -54,10 +60,8 @@ void ArgMinMaxStateBase::DestroyValue(Vector *&value) { } template <> -void ArgMinMaxStateBase::AssignValue(string_t &target, string_t new_value, bool is_initialized) { - if (is_initialized) { - DestroyValue(target); - } +void ArgMinMaxStateBase::AssignValue(string_t &target, string_t new_value) { + DestroyValue(target); if (new_value.IsInlined()) { target = new_value; } else { @@ -97,7 +101,7 @@ struct ArgMinMaxState : public ArgMinMaxStateBase { } }; -template +template struct ArgMinMaxBase { template @@ -110,22 +114,36 @@ struct ArgMinMaxBase { state.~STATE(); } + template + static void Assign(STATE &state, const A_TYPE &x, const B_TYPE &y, const bool x_null) { + if (IGNORE_NULL) { + STATE::template AssignValue(state.arg, x); + STATE::template AssignValue(state.value, y); + } else { + state.arg_null = x_null; + if (!state.arg_null) { + STATE::template AssignValue(state.arg, x); + } + STATE::template AssignValue(state.value, y); + } + } + template - static void Operation(STATE &state, const A_TYPE &x, const B_TYPE &y, AggregateBinaryInput &) { + static void Operation(STATE &state, const A_TYPE &x, const B_TYPE &y, AggregateBinaryInput &binary) { if (!state.is_initialized) { - STATE::template AssignValue(state.arg, x, false); - STATE::template AssignValue(state.value, y, false); - state.is_initialized = true; + if (IGNORE_NULL || binary.right_mask.RowIsValid(binary.ridx)) { + Assign(state, x, y, !binary.left_mask.RowIsValid(binary.lidx)); + state.is_initialized = true; + } } else { - OP::template Execute(state, x, y); + OP::template Execute(state, x, y, binary); } } template - static void Execute(STATE &state, A_TYPE x_data, B_TYPE y_data) { - if (COMPARATOR::Operation(y_data, state.value)) { - STATE::template AssignValue(state.arg, x_data, true); - STATE::template AssignValue(state.value, y_data, true); + static void Execute(STATE &state, A_TYPE x_data, B_TYPE y_data, AggregateBinaryInput &binary) { + if ((IGNORE_NULL || binary.right_mask.RowIsValid(binary.ridx)) && COMPARATOR::Operation(y_data, state.value)) { + Assign(state, x_data, y_data, !binary.left_mask.RowIsValid(binary.lidx)); } } @@ -135,15 +153,14 @@ struct ArgMinMaxBase { return; } if (!target.is_initialized || COMPARATOR::Operation(source.value, target.value)) { - STATE::template AssignValue(target.arg, source.arg, target.is_initialized); - STATE::template AssignValue(target.value, source.value, target.is_initialized); + Assign(target, source.arg, source.value, source.arg_null); target.is_initialized = true; } } template static void Finalize(STATE &state, T &target, AggregateFinalizeData &finalize_data) { - if (!state.is_initialized) { + if (!state.is_initialized || state.arg_null) { finalize_data.ReturnNull(); } else { STATE::template ReadValue(finalize_data.result, state.arg, target); @@ -151,31 +168,32 @@ struct ArgMinMaxBase { } static bool IgnoreNull() { - return true; + return IGNORE_NULL; } static unique_ptr Bind(ClientContext &context, AggregateFunction &function, vector> &arguments) { - auto result_type = - BoundComparisonExpression::BindComparison(arguments[0]->return_type, arguments[1]->return_type); - ExpressionBinder::PushCollation(context, arguments[1], result_type, false); + ExpressionBinder::PushCollation(context, arguments[1], arguments[1]->return_type, false); function.arguments[0] = arguments[0]->return_type; function.return_type = arguments[0]->return_type; return nullptr; } }; -template -struct VectorArgMinMaxBase : ArgMinMaxBase { +template +struct VectorArgMinMaxBase : ArgMinMaxBase { template - static void AssignVector(STATE &state, Vector &arg, const idx_t idx) { - if (!state.is_initialized) { - state.arg = new Vector(arg.GetType()); + static void AssignVector(STATE &state, Vector &arg, bool arg_null, const idx_t idx) { + if (!state.arg) { + state.arg = new Vector(arg.GetType(), 1); state.arg->SetVectorType(VectorType::CONSTANT_VECTOR); } - sel_t selv = idx; - SelectionVector sel(&selv); - VectorOperations::Copy(arg, *state.arg, sel, 1, 0, 0); + state.arg_null = arg_null; + if (!arg_null) { + sel_t selv = idx; + SelectionVector sel(&selv); + VectorOperations::Copy(arg, *state.arg, sel, 1, 0, 0); + } } template @@ -201,16 +219,22 @@ struct VectorArgMinMaxBase : ArgMinMaxBase { } const auto bval = bys[bidx]; + const auto aidx = adata.sel->get_index(i); + const auto arg_null = !adata.validity.RowIsValid(aidx); + if (IGNORE_NULL && arg_null) { + continue; + } + const auto sidx = sdata.sel->get_index(i); auto &state = *states[sidx]; if (!state.is_initialized) { - STATE::template AssignValue(state.value, bval, false); - AssignVector(state, arg, i); + STATE::template AssignValue(state.value, bval); + AssignVector(state, arg, arg_null, i); state.is_initialized = true; } else if (COMPARATOR::template Operation(bval, state.value)) { - STATE::template AssignValue(state.value, bval, true); - AssignVector(state, arg, i); + STATE::template AssignValue(state.value, bval); + AssignVector(state, arg, arg_null, i); } } } @@ -221,15 +245,15 @@ struct VectorArgMinMaxBase : ArgMinMaxBase { return; } if (!target.is_initialized || COMPARATOR::Operation(source.value, target.value)) { - STATE::template AssignValue(target.value, source.value, target.is_initialized); - AssignVector(target, *source.arg, 0); + STATE::template AssignValue(target.value, source.value); + AssignVector(target, *source.arg, source.arg_null, 0); target.is_initialized = true; } } template static void Finalize(STATE &state, AggregateFinalizeData &finalize_data) { - if (!state.is_initialized) { + if (!state.is_initialized || state.arg_null) { finalize_data.ReturnNull(); } else { VectorOperations::Copy(*state.arg, finalize_data.result, 1, 0, finalize_data.result_idx); @@ -269,16 +293,19 @@ AggregateFunction GetVectorArgMinMaxFunctionBy(const LogicalType &by_type, const } } +static const vector ArgMaxByTypes() { + vector types = {LogicalType::INTEGER, LogicalType::BIGINT, LogicalType::DOUBLE, + LogicalType::VARCHAR, LogicalType::DATE, LogicalType::TIMESTAMP, + LogicalType::TIMESTAMP_TZ, LogicalType::BLOB}; + return types; +} + template void AddVectorArgMinMaxFunctionBy(AggregateFunctionSet &fun, const LogicalType &type) { - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::INTEGER, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::BIGINT, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::DOUBLE, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::VARCHAR, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::DATE, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::TIMESTAMP, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::TIMESTAMP_TZ, type)); - fun.AddFunction(GetVectorArgMinMaxFunctionBy(LogicalType::BLOB, type)); + auto by_types = ArgMaxByTypes(); + for (const auto &by_type : by_types) { + fun.AddFunction(GetVectorArgMinMaxFunctionBy(by_type, type)); + } } template @@ -312,19 +339,48 @@ AggregateFunction GetArgMinMaxFunctionBy(const LogicalType &by_type, const Logic template void AddArgMinMaxFunctionBy(AggregateFunctionSet &fun, const LogicalType &type) { - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::INTEGER, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::BIGINT, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::DOUBLE, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::VARCHAR, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::DATE, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::TIMESTAMP, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::TIMESTAMP_TZ, type)); - fun.AddFunction(GetArgMinMaxFunctionBy(LogicalType::BLOB, type)); + auto by_types = ArgMaxByTypes(); + for (const auto &by_type : by_types) { + fun.AddFunction(GetArgMinMaxFunctionBy(by_type, type)); + } +} + +template +static AggregateFunction GetDecimalArgMinMaxFunction(const LogicalType &by_type, const LogicalType &type) { + D_ASSERT(type.id() == LogicalTypeId::DECIMAL); + switch (type.InternalType()) { + case PhysicalType::INT16: + return GetArgMinMaxFunctionBy(by_type, type); + case PhysicalType::INT32: + return GetArgMinMaxFunctionBy(by_type, type); + case PhysicalType::INT64: + return GetArgMinMaxFunctionBy(by_type, type); + default: + return GetArgMinMaxFunctionBy(by_type, type); + } +} + +template +static unique_ptr BindDecimalArgMinMax(ClientContext &context, AggregateFunction &function, + vector> &arguments) { + auto decimal_type = arguments[0]->return_type; + auto by_type = arguments[1]->return_type; + auto name = std::move(function.name); + function = GetDecimalArgMinMaxFunction(by_type, decimal_type); + function.name = std::move(name); + function.return_type = decimal_type; + return nullptr; +} + +template +void AddDecimalArgMinMaxFunctionBy(AggregateFunctionSet &fun, const LogicalType &by_type) { + fun.AddFunction(AggregateFunction({LogicalTypeId::DECIMAL, by_type}, LogicalTypeId::DECIMAL, nullptr, nullptr, + nullptr, nullptr, nullptr, nullptr, BindDecimalArgMinMax)); } -template +template static void AddArgMinMaxFunctions(AggregateFunctionSet &fun) { - using OP = ArgMinMaxBase; + using OP = ArgMinMaxBase; AddArgMinMaxFunctionBy(fun, LogicalType::INTEGER); AddArgMinMaxFunctionBy(fun, LogicalType::BIGINT); AddArgMinMaxFunctionBy(fun, LogicalType::DOUBLE); @@ -334,19 +390,36 @@ static void AddArgMinMaxFunctions(AggregateFunctionSet &fun) { AddArgMinMaxFunctionBy(fun, LogicalType::TIMESTAMP_TZ); AddArgMinMaxFunctionBy(fun, LogicalType::BLOB); - using VECTOR_OP = VectorArgMinMaxBase; + auto by_types = ArgMaxByTypes(); + for (const auto &by_type : by_types) { + AddDecimalArgMinMaxFunctionBy(fun, by_type); + } + + using VECTOR_OP = VectorArgMinMaxBase; AddVectorArgMinMaxFunctionBy(fun, LogicalType::ANY); } AggregateFunctionSet ArgMinFun::GetFunctions() { AggregateFunctionSet fun; - AddArgMinMaxFunctions(fun); + AddArgMinMaxFunctions(fun); return fun; } AggregateFunctionSet ArgMaxFun::GetFunctions() { AggregateFunctionSet fun; - AddArgMinMaxFunctions(fun); + AddArgMinMaxFunctions(fun); + return fun; +} + +AggregateFunctionSet ArgMinNullFun::GetFunctions() { + AggregateFunctionSet fun; + AddArgMinMaxFunctions(fun); + return fun; +} + +AggregateFunctionSet ArgMaxNullFun::GetFunctions() { + AggregateFunctionSet fun; + AddArgMinMaxFunctions(fun); return fun; } diff --git a/src/duckdb/src/core_functions/aggregate/distributive/bitagg.cpp b/src/duckdb/src/core_functions/aggregate/distributive/bitagg.cpp index 52b671d96..18d7a7255 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/bitagg.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/bitagg.cpp @@ -35,6 +35,8 @@ static AggregateFunction GetBitfieldUnaryAggregate(LogicalType type) { return AggregateFunction::UnaryAggregate, uint32_t, uint32_t, OP>(type, type); case LogicalTypeId::UBIGINT: return AggregateFunction::UnaryAggregate, uint64_t, uint64_t, OP>(type, type); + case LogicalTypeId::UHUGEINT: + return AggregateFunction::UnaryAggregate, uhugeint_t, uhugeint_t, OP>(type, type); default: throw InternalException("Unimplemented bitfield type for unary aggregate"); } diff --git a/src/duckdb/src/core_functions/aggregate/distributive/bitstring_agg.cpp b/src/duckdb/src/core_functions/aggregate/distributive/bitstring_agg.cpp index 303021bd3..e78b228f2 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/bitstring_agg.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/bitstring_agg.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/vector_operations/aggregate_executor.hpp" #include "duckdb/common/types/bit.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/storage/statistics/base_statistics.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/common/types/cast_helpers.hpp" @@ -183,6 +184,29 @@ idx_t BitStringAggOperation::GetRange(hugeint_t min, hugeint_t max) { return range; } +template <> +void BitStringAggOperation::Execute(BitAggState &state, uhugeint_t input, uhugeint_t min) { + idx_t val; + if (Uhugeint::TryCast(input - min, val)) { + Bit::SetBit(state.value, val, 1); + } else { + throw OutOfRangeException("Range too large for bitstring aggregation"); + } +} + +template <> +idx_t BitStringAggOperation::GetRange(uhugeint_t min, uhugeint_t max) { + uhugeint_t result; + if (!TrySubtractOperator::Operation(max, min, result)) { + return NumericLimits::Maximum(); + } + idx_t range; + if (!Uhugeint::TryCast(result + 1, range)) { + return NumericLimits::Maximum(); + } + return range; +} + unique_ptr BitstringPropagateStats(ClientContext &context, BoundAggregateExpression &expr, AggregateStatisticsInput &input) { @@ -253,6 +277,9 @@ void GetBitStringAggregate(const LogicalType &type, AggregateFunctionSet &bitstr case LogicalType::UBIGINT: { return BindBitString(bitstring_agg, type.id()); } + case LogicalType::UHUGEINT: { + return BindBitString(bitstring_agg, type.id()); + } default: throw InternalException("Unimplemented bitstring aggregate"); } diff --git a/src/duckdb/src/core_functions/aggregate/distributive/entropy.cpp b/src/duckdb/src/core_functions/aggregate/distributive/entropy.cpp index b965b8119..e66f30783 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/entropy.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/entropy.cpp @@ -146,10 +146,11 @@ AggregateFunction GetEntropyFunctionInternal(PhysicalType type) { case PhysicalType::DOUBLE: return AggregateFunction::UnaryAggregateDestructor, double, double, EntropyFunction>( LogicalType::DOUBLE, LogicalType::DOUBLE); - case PhysicalType::VARCHAR: + case PhysicalType::VARCHAR: { return AggregateFunction::UnaryAggregateDestructor, string_t, double, - EntropyFunctionString>(LogicalType::VARCHAR, - LogicalType::DOUBLE); + EntropyFunctionString>( + LogicalType::ANY_PARAMS(LogicalType::VARCHAR, 150), LogicalType::DOUBLE); + } default: throw InternalException("Unimplemented approximate_count aggregate"); diff --git a/src/duckdb/src/core_functions/aggregate/distributive/kurtosis.cpp b/src/duckdb/src/core_functions/aggregate/distributive/kurtosis.cpp index 063408e0e..bca51bd16 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/kurtosis.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/kurtosis.cpp @@ -14,6 +14,11 @@ struct KurtosisState { double sum_four; }; +struct KurtosisFlagBiasCorrection {}; + +struct KurtosisFlagNoBiasCorrection {}; + +template struct KurtosisOperation { template static void Initialize(STATE &state) { @@ -53,7 +58,11 @@ struct KurtosisOperation { template static void Finalize(STATE &state, TARGET_TYPE &target, AggregateFinalizeData &finalize_data) { auto n = (double)state.n; - if (n <= 3) { + if (n <= 1) { + finalize_data.ReturnNull(); + return; + } + if (std::is_same::value && n <= 3) { finalize_data.ReturnNull(); return; } @@ -70,11 +79,15 @@ struct KurtosisOperation { 6 * state.sum_sqr * state.sum * state.sum * temp * temp - 3 * pow(state.sum, 4) * pow(temp, 3)); double m2 = temp * (state.sum_sqr - state.sum * state.sum * temp); - if (m2 <= 0 || ((n - 2) * (n - 3)) == 0) { // m2 shouldn't be below 0 but floating points are weird + if (m2 <= 0) { // m2 shouldn't be below 0 but floating points are weird finalize_data.ReturnNull(); return; } - target = (n - 1) * ((n + 1) * m4 / (m2 * m2) - 3 * (n - 1)) / ((n - 2) * (n - 3)); + if (std::is_same::value) { + target = m4 / (m2 * m2) - 3; + } else { + target = (n - 1) * ((n + 1) * m4 / (m2 * m2) - 3 * (n - 1)) / ((n - 2) * (n - 3)); + } if (!Value::DoubleIsFinite(target)) { throw OutOfRangeException("Kurtosis is out of range!"); } @@ -86,8 +99,15 @@ struct KurtosisOperation { }; AggregateFunction KurtosisFun::GetFunction() { - return AggregateFunction::UnaryAggregate(LogicalType::DOUBLE, - LogicalType::DOUBLE); + return AggregateFunction::UnaryAggregate>(LogicalType::DOUBLE, + LogicalType::DOUBLE); +} + +AggregateFunction KurtosisPopFun::GetFunction() { + return AggregateFunction::UnaryAggregate>(LogicalType::DOUBLE, + LogicalType::DOUBLE); } } // namespace duckdb diff --git a/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp b/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp index 661d3938c..f3ef05885 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/minmax.cpp @@ -40,6 +40,8 @@ static AggregateFunction GetUnaryAggregate(LogicalType type) { return AggregateFunction::UnaryAggregate, uint64_t, uint64_t, OP>(type, type); case PhysicalType::INT128: return AggregateFunction::UnaryAggregate, hugeint_t, hugeint_t, OP>(type, type); + case PhysicalType::UINT128: + return AggregateFunction::UnaryAggregate, uhugeint_t, uhugeint_t, OP>(type, type); case PhysicalType::FLOAT: return AggregateFunction::UnaryAggregate, float, float, OP>(type, type); case PhysicalType::DOUBLE: @@ -264,6 +266,8 @@ static bool TemplatedOptimumValue(Vector &left, idx_t lidx, idx_t lcount, Vector return TemplatedOptimumType(left, lidx, lcount, right, ridx, rcount); case PhysicalType::INT128: return TemplatedOptimumType(left, lidx, lcount, right, ridx, rcount); + case PhysicalType::UINT128: + return TemplatedOptimumType(left, lidx, lcount, right, ridx, rcount); case PhysicalType::FLOAT: return TemplatedOptimumType(left, lidx, lcount, right, ridx, rcount); case PhysicalType::DOUBLE: @@ -602,10 +606,11 @@ unique_ptr BindMinMax(ClientContext &context, AggregateFunction &f FunctionBinder function_binder(context); vector types {arguments[0]->return_type, arguments[0]->return_type}; - string error; + ErrorData error; idx_t best_function = function_binder.BindFunction(func_entry.name, func_entry.functions, types, error); if (best_function == DConstants::INVALID_INDEX) { - throw BinderException(string("Fail to find corresponding function for collation min/max: ") + error); + throw BinderException(string("Fail to find corresponding function for collation min/max: ") + + error.Message()); } function = func_entry.functions.GetFunctionByOffset(best_function); diff --git a/src/duckdb/src/core_functions/aggregate/distributive/string_agg.cpp b/src/duckdb/src/core_functions/aggregate/distributive/string_agg.cpp index b09c52fc0..bd3e95fb6 100644 --- a/src/duckdb/src/core_functions/aggregate/distributive/string_agg.cpp +++ b/src/duckdb/src/core_functions/aggregate/distributive/string_agg.cpp @@ -155,7 +155,8 @@ unique_ptr StringAggDeserialize(Deserializer &deserializer, Aggreg AggregateFunctionSet StringAggFun::GetFunctions() { AggregateFunctionSet string_agg; AggregateFunction string_agg_param( - {LogicalType::VARCHAR}, LogicalType::VARCHAR, AggregateFunction::StateSize, + {LogicalType::ANY_PARAMS(LogicalType::VARCHAR)}, LogicalType::VARCHAR, + AggregateFunction::StateSize, AggregateFunction::StateInitialize, AggregateFunction::UnaryScatterUpdate, AggregateFunction::StateCombine, diff --git a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp index 14d0904bf..ab6a76bce 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/mode.cpp @@ -3,6 +3,7 @@ // NULL values are ignored. If all the values are NULL, or there are 0 rows, then the function returns NULL. #include "duckdb/common/exception.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/operator/comparison_operators.hpp" #include "duckdb/core_functions/aggregate/holistic_functions.hpp" @@ -29,6 +30,13 @@ struct hash { } }; +template <> +struct hash { + inline size_t operator()(const duckdb::uhugeint_t &val) const { + return hash {}(val.upper) ^ hash {}(val.lower); + } +}; + } // namespace std namespace duckdb { @@ -304,7 +312,8 @@ template ; using OP = ModeFunction; - auto func = AggregateFunction::UnaryAggregateDestructor(type, type); + auto return_type = type.id() == LogicalTypeId::ANY ? LogicalType::VARCHAR : type; + auto func = AggregateFunction::UnaryAggregateDestructor(type, return_type); func.window = AggregateFunction::UnaryWindow; return func; } @@ -329,6 +338,8 @@ AggregateFunction GetModeAggregate(const LogicalType &type) { return GetTypedModeFunction(type); case PhysicalType::INT128: return GetTypedModeFunction(type); + case PhysicalType::UINT128: + return GetTypedModeFunction(type); case PhysicalType::FLOAT: return GetTypedModeFunction(type); @@ -339,7 +350,8 @@ AggregateFunction GetModeAggregate(const LogicalType &type) { return GetTypedModeFunction(type); case PhysicalType::VARCHAR: - return GetTypedModeFunction(type); + return GetTypedModeFunction( + LogicalType::ANY_PARAMS(LogicalType::VARCHAR, 150)); default: throw NotImplementedException("Unimplemented mode aggregate"); diff --git a/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp b/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp index 2a21079d8..a9af11250 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/quantile.cpp @@ -6,7 +6,7 @@ #include "duckdb/common/operator/cast_operators.hpp" #include "duckdb/common/operator/abs.hpp" #include "duckdb/common/operator/multiply.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/queue.hpp" #include "duckdb/common/serializer/serializer.hpp" @@ -907,7 +907,8 @@ struct QuantileOperation { template static AggregateFunction QuantileListAggregate(const LogicalType &input_type, const LogicalType &child_type) { // NOLINT - LogicalType result_type = LogicalType::LIST(child_type); + LogicalType result_type = + LogicalType::LIST(child_type.id() == LogicalTypeId::ANY ? LogicalType::VARCHAR : child_type); return AggregateFunction( {input_type}, result_type, AggregateFunction::StateSize, AggregateFunction::StateInitialize, AggregateFunction::UnaryScatterUpdate, AggregateFunction::StateCombine, @@ -969,7 +970,8 @@ template AggregateFunction GetTypedDiscreteQuantileAggregateFunction(const LogicalType &type) { using STATE = QuantileState; using OP = QuantileScalarOperation; - auto fun = AggregateFunction::UnaryAggregateDestructor(type, type); + auto return_type = type.id() == LogicalTypeId::ANY ? LogicalType::VARCHAR : type; + auto fun = AggregateFunction::UnaryAggregateDestructor(type, return_type); fun.window = AggregateFunction::UnaryWindow; fun.window_init = OP::WindowInit; return fun; @@ -1014,8 +1016,7 @@ AggregateFunction GetDiscreteQuantileAggregateFunction(const LogicalType &type) return GetTypedDiscreteQuantileAggregateFunction(type); case LogicalTypeId::INTERVAL: return GetTypedDiscreteQuantileAggregateFunction(type); - - case LogicalTypeId::VARCHAR: + case LogicalTypeId::ANY: return GetTypedDiscreteQuantileAggregateFunction(type); default: @@ -1137,7 +1138,7 @@ AggregateFunction GetDiscreteQuantileListAggregateFunction(const LogicalType &ty return GetTypedDiscreteQuantileListAggregateFunction(type); case LogicalTypeId::INTERVAL: return GetTypedDiscreteQuantileListAggregateFunction(type); - case LogicalTypeId::VARCHAR: + case LogicalTypeId::ANY: return GetTypedDiscreteQuantileListAggregateFunction(type); default: throw NotImplementedException("Unimplemented discrete quantile list aggregate"); @@ -1223,7 +1224,6 @@ AggregateFunction GetContinuousQuantileListAggregateFunction(const LogicalType & return GetTypedContinuousQuantileListAggregateFunction(type, LogicalType::DOUBLE); case LogicalTypeId::HUGEINT: return GetTypedContinuousQuantileListAggregateFunction(type, LogicalType::DOUBLE); - case LogicalTypeId::FLOAT: return GetTypedContinuousQuantileListAggregateFunction(type, type); case LogicalTypeId::DOUBLE: @@ -1241,8 +1241,6 @@ AggregateFunction GetContinuousQuantileListAggregateFunction(const LogicalType & default: throw NotImplementedException("Unimplemented discrete quantile DECIMAL list aggregate"); } - break; - case LogicalTypeId::DATE: return GetTypedContinuousQuantileListAggregateFunction(type, LogicalType::TIMESTAMP); case LogicalTypeId::TIMESTAMP: @@ -1251,7 +1249,6 @@ AggregateFunction GetContinuousQuantileListAggregateFunction(const LogicalType & case LogicalTypeId::TIME: case LogicalTypeId::TIME_TZ: return GetTypedContinuousQuantileListAggregateFunction(type, type); - default: throw NotImplementedException("Unimplemented discrete quantile list aggregate"); } @@ -1585,6 +1582,7 @@ static bool CanInterpolate(const LogicalType &type) { switch (type.id()) { case LogicalTypeId::INTERVAL: case LogicalTypeId::VARCHAR: + case LogicalTypeId::ANY: return false; default: return true; @@ -1657,10 +1655,13 @@ AggregateFunction GetQuantileDecimalAggregate(const vector &argumen } vector GetQuantileTypes() { - return {LogicalType::TINYINT, LogicalType::SMALLINT, LogicalType::INTEGER, LogicalType::BIGINT, - LogicalType::HUGEINT, LogicalType::FLOAT, LogicalType::DOUBLE, LogicalType::DATE, - LogicalType::TIMESTAMP, LogicalType::TIME, LogicalType::TIMESTAMP_TZ, LogicalType::TIME_TZ, - LogicalType::INTERVAL, LogicalType::VARCHAR}; + return {LogicalType::TINYINT, LogicalType::SMALLINT, + LogicalType::INTEGER, LogicalType::BIGINT, + LogicalType::HUGEINT, LogicalType::FLOAT, + LogicalType::DOUBLE, LogicalType::DATE, + LogicalType::TIMESTAMP, LogicalType::TIME, + LogicalType::TIMESTAMP_TZ, LogicalType::TIME_TZ, + LogicalType::INTERVAL, LogicalType::ANY_PARAMS(LogicalType::VARCHAR, 150)}; } AggregateFunctionSet MedianFun::GetFunctions() { diff --git a/src/duckdb/src/core_functions/aggregate/holistic/reservoir_quantile.cpp b/src/duckdb/src/core_functions/aggregate/holistic/reservoir_quantile.cpp index 8945948e1..7da2cdbee 100644 --- a/src/duckdb/src/core_functions/aggregate/holistic/reservoir_quantile.cpp +++ b/src/duckdb/src/core_functions/aggregate/holistic/reservoir_quantile.cpp @@ -32,7 +32,7 @@ struct ReservoirQuantileState { } void ReplaceElement(T &input) { - v[r_samp->min_entry] = input; + v[r_samp->min_weighted_entry_index] = input; r_samp->ReplaceElement(); } @@ -41,8 +41,8 @@ struct ReservoirQuantileState { v[pos++] = element; r_samp->InitializeReservoir(pos, len); } else { - D_ASSERT(r_samp->next_index >= r_samp->current_count); - if (r_samp->next_index == r_samp->current_count) { + D_ASSERT(r_samp->next_index_to_sample >= r_samp->num_entries_to_skip_b4_next_sample); + if (r_samp->next_index_to_sample == r_samp->num_entries_to_skip_b4_next_sample) { ReplaceElement(element); } } diff --git a/src/duckdb/src/core_functions/aggregate/nested/histogram.cpp b/src/duckdb/src/core_functions/aggregate/nested/histogram.cpp index b50c535e2..2f7414340 100644 --- a/src/duckdb/src/core_functions/aggregate/nested/histogram.cpp +++ b/src/duckdb/src/core_functions/aggregate/nested/histogram.cpp @@ -155,8 +155,8 @@ unique_ptr HistogramBindFunction(ClientContext &context, Aggregate arguments[0]->return_type.id() == LogicalTypeId::MAP) { throw NotImplementedException("Unimplemented type for histogram %s", arguments[0]->return_type.ToString()); } - - auto struct_type = LogicalType::MAP(arguments[0]->return_type, LogicalType::UBIGINT); + auto child_type = function.arguments[0].id() == LogicalTypeId::ANY ? LogicalType::VARCHAR : function.arguments[0]; + auto struct_type = LogicalType::MAP(child_type, LogicalType::UBIGINT); function.return_type = struct_type; return make_uniq(function.return_type); @@ -176,7 +176,6 @@ static AggregateFunction GetHistogramFunction(const LogicalType &type) { template AggregateFunction GetMapType(const LogicalType &type) { - if (IS_ORDERED) { return GetHistogramFunction(type); } @@ -185,48 +184,47 @@ AggregateFunction GetMapType(const LogicalType &type) { template AggregateFunction GetHistogramFunction(const LogicalType &type) { - switch (type.id()) { - case LogicalType::BOOLEAN: + case LogicalTypeId::BOOLEAN: return GetMapType(type); - case LogicalType::UTINYINT: + case LogicalTypeId::UTINYINT: return GetMapType(type); - case LogicalType::USMALLINT: + case LogicalTypeId::USMALLINT: return GetMapType(type); - case LogicalType::UINTEGER: + case LogicalTypeId::UINTEGER: return GetMapType(type); - case LogicalType::UBIGINT: + case LogicalTypeId::UBIGINT: return GetMapType(type); - case LogicalType::TINYINT: + case LogicalTypeId::TINYINT: return GetMapType(type); - case LogicalType::SMALLINT: + case LogicalTypeId::SMALLINT: return GetMapType(type); - case LogicalType::INTEGER: + case LogicalTypeId::INTEGER: return GetMapType(type); - case LogicalType::BIGINT: + case LogicalTypeId::BIGINT: return GetMapType(type); - case LogicalType::FLOAT: + case LogicalTypeId::FLOAT: return GetMapType(type); - case LogicalType::DOUBLE: + case LogicalTypeId::DOUBLE: return GetMapType(type); - case LogicalType::VARCHAR: - return GetMapType(type); - case LogicalType::TIMESTAMP: + case LogicalTypeId::TIMESTAMP: return GetMapType(type); - case LogicalType::TIMESTAMP_TZ: + case LogicalTypeId::TIMESTAMP_TZ: return GetMapType(type); - case LogicalType::TIMESTAMP_S: + case LogicalTypeId::TIMESTAMP_SEC: return GetMapType(type); - case LogicalType::TIMESTAMP_MS: + case LogicalTypeId::TIMESTAMP_MS: return GetMapType(type); - case LogicalType::TIMESTAMP_NS: + case LogicalTypeId::TIMESTAMP_NS: return GetMapType(type); - case LogicalType::TIME: + case LogicalTypeId::TIME: return GetMapType(type); - case LogicalType::TIME_TZ: + case LogicalTypeId::TIME_TZ: return GetMapType(type); - case LogicalType::DATE: + case LogicalTypeId::DATE: return GetMapType(type); + case LogicalTypeId::ANY: + return GetMapType(type); default: throw InternalException("Unimplemented histogram aggregate"); } @@ -245,7 +243,6 @@ AggregateFunctionSet HistogramFun::GetFunctions() { fun.AddFunction(GetHistogramFunction<>(LogicalType::BIGINT)); fun.AddFunction(GetHistogramFunction<>(LogicalType::FLOAT)); fun.AddFunction(GetHistogramFunction<>(LogicalType::DOUBLE)); - fun.AddFunction(GetHistogramFunction<>(LogicalType::VARCHAR)); fun.AddFunction(GetHistogramFunction<>(LogicalType::TIMESTAMP)); fun.AddFunction(GetHistogramFunction<>(LogicalType::TIMESTAMP_TZ)); fun.AddFunction(GetHistogramFunction<>(LogicalType::TIMESTAMP_S)); @@ -254,6 +251,7 @@ AggregateFunctionSet HistogramFun::GetFunctions() { fun.AddFunction(GetHistogramFunction<>(LogicalType::TIME)); fun.AddFunction(GetHistogramFunction<>(LogicalType::TIME_TZ)); fun.AddFunction(GetHistogramFunction<>(LogicalType::DATE)); + fun.AddFunction(GetHistogramFunction<>(LogicalType::ANY_PARAMS(LogicalType::VARCHAR))); return fun; } diff --git a/src/duckdb/src/core_functions/aggregate/nested/list.cpp b/src/duckdb/src/core_functions/aggregate/nested/list.cpp index e9a39593c..1895cd27b 100644 --- a/src/duckdb/src/core_functions/aggregate/nested/list.cpp +++ b/src/duckdb/src/core_functions/aggregate/nested/list.cpp @@ -67,7 +67,9 @@ static void ListUpdateFunction(Vector inputs[], AggregateInputData &aggr_input_d } } -static void ListCombineFunction(Vector &states_vector, Vector &combined, AggregateInputData &, idx_t count) { +static void ListAbsorbFunction(Vector &states_vector, Vector &combined, AggregateInputData &aggr_input_data, + idx_t count) { + D_ASSERT(aggr_input_data.combine_type == AggregateCombineType::ALLOW_DESTRUCTIVE); UnifiedVectorFormat states_data; states_vector.ToUnifiedFormat(count, states_data); @@ -147,49 +149,38 @@ static void ListFinalize(Vector &states_vector, AggregateInputData &aggr_input_d ListVector::SetListSize(result, total_len); } -static void ListWindow(AggregateInputData &aggr_input_data, const WindowPartitionInput &partition, - const_data_ptr_t g_state, data_ptr_t l_state, const SubFrames &frames, Vector &result, - idx_t rid) { +static void ListCombineFunction(Vector &states_vector, Vector &combined, AggregateInputData &aggr_input_data, + idx_t count) { - auto &list_bind_data = aggr_input_data.bind_data->Cast(); - LinkedList linked_list; + // Can we use destructive combining? + if (aggr_input_data.combine_type == AggregateCombineType::ALLOW_DESTRUCTIVE) { + ListAbsorbFunction(states_vector, combined, aggr_input_data, count); + return; + } + + UnifiedVectorFormat states_data; + states_vector.ToUnifiedFormat(count, states_data); + auto states_ptr = UnifiedVectorFormat::GetData(states_data); + auto combined_ptr = FlatVector::GetData(combined); - // UPDATE step + auto &list_bind_data = aggr_input_data.bind_data->Cast(); + auto result_type = ListType::GetChildType(list_bind_data.stype); - D_ASSERT(partition.input_count == 1); - // FIXME: We are modifying the window operator's data here - auto &input = const_cast(partition.inputs[0]); + for (idx_t i = 0; i < count; i++) { + auto &source = *states_ptr[states_data.sel->get_index(i)]; + auto &target = *combined_ptr[i]; - // FIXME: we unify more values than necessary (count is frame.end) - const auto count = frames.back().end; + const auto entry_count = source.linked_list.total_capacity; + Vector input(result_type, source.linked_list.total_capacity); + list_bind_data.functions.BuildListVector(source.linked_list, input, 0); - RecursiveUnifiedVectorFormat input_data; - Vector::RecursiveToUnifiedFormat(input, count, input_data); + RecursiveUnifiedVectorFormat input_data; + Vector::RecursiveToUnifiedFormat(input, entry_count, input_data); - 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); + for (idx_t entry_idx = 0; entry_idx < entry_count; ++entry_idx) { + list_bind_data.functions.AppendRow(aggr_input_data.allocator, target.linked_list, input_data, entry_idx); } } - - // FINALIZE step - - D_ASSERT(result.GetType().id() == LogicalTypeId::LIST); - auto result_data = FlatVector::GetData(result); - size_t total_len = ListVector::GetListSize(result); - - // set the length and offset of this list in the result vector - result_data[rid].offset = total_len; - result_data[rid].length = linked_list.total_capacity; - total_len += linked_list.total_capacity; - - // reserve capacity, then copy over the data to the child vector - ListVector::Reserve(result, total_len); - auto &result_child = ListVector::GetEntry(result); - idx_t offset = result_data[rid].offset; - list_bind_data.functions.BuildListVector(linked_list, result_child, offset); - - ListVector::SetListSize(result, total_len); } unique_ptr ListBindFunction(ClientContext &context, AggregateFunction &function, @@ -208,10 +199,12 @@ unique_ptr ListBindFunction(ClientContext &context, AggregateFunct } AggregateFunction ListFun::GetFunction() { - return AggregateFunction({LogicalType::ANY}, LogicalTypeId::LIST, AggregateFunction::StateSize, - AggregateFunction::StateInitialize, ListUpdateFunction, - ListCombineFunction, ListFinalize, nullptr, ListBindFunction, nullptr, nullptr, - ListWindow); + auto func = + AggregateFunction({LogicalType::ANY}, LogicalTypeId::LIST, AggregateFunction::StateSize, + AggregateFunction::StateInitialize, ListUpdateFunction, + ListCombineFunction, ListFinalize, nullptr, ListBindFunction, nullptr, nullptr, nullptr); + + return func; } } // namespace duckdb diff --git a/src/duckdb/src/core_functions/function_list.cpp b/src/duckdb/src/core_functions/function_list.cpp index 80a70d524..5fad61613 100644 --- a/src/duckdb/src/core_functions/function_list.cpp +++ b/src/duckdb/src/core_functions/function_list.cpp @@ -14,6 +14,7 @@ #include "duckdb/core_functions/scalar/math_functions.hpp" #include "duckdb/core_functions/scalar/operators_functions.hpp" #include "duckdb/core_functions/scalar/random_functions.hpp" +#include "duckdb/core_functions/scalar/secret_functions.hpp" #include "duckdb/core_functions/scalar/string_functions.hpp" #include "duckdb/core_functions/scalar/struct_functions.hpp" #include "duckdb/core_functions/scalar/union_functions.hpp" @@ -67,7 +68,9 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_AGGREGATE_FUNCTION_SET(ApproxCountDistinctFun), DUCKDB_AGGREGATE_FUNCTION_SET(ApproxQuantileFun), DUCKDB_AGGREGATE_FUNCTION_SET(ArgMaxFun), + DUCKDB_AGGREGATE_FUNCTION_SET(ArgMaxNullFun), DUCKDB_AGGREGATE_FUNCTION_SET(ArgMinFun), + DUCKDB_AGGREGATE_FUNCTION_SET(ArgMinNullFun), DUCKDB_AGGREGATE_FUNCTION_SET_ALIAS(ArgmaxFun), DUCKDB_AGGREGATE_FUNCTION_SET_ALIAS(ArgminFun), DUCKDB_AGGREGATE_FUNCTION_ALIAS(ArrayAggFun), @@ -82,6 +85,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayFilterFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArrayGradeUpFun), DUCKDB_SCALAR_FUNCTION_SET(ArrayInnerProductFun), + DUCKDB_SCALAR_FUNCTION_ALIAS(ArrayReduceFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArrayReverseSortFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArraySliceFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(ArraySortFun), @@ -116,6 +120,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(CotFun), DUCKDB_AGGREGATE_FUNCTION(CovarPopFun), DUCKDB_AGGREGATE_FUNCTION(CovarSampFun), + DUCKDB_SCALAR_FUNCTION(CreateSortKeyFun), DUCKDB_SCALAR_FUNCTION(CurrentDatabaseFun), DUCKDB_SCALAR_FUNCTION(CurrentDateFun), DUCKDB_SCALAR_FUNCTION(CurrentQueryFun), @@ -198,6 +203,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_SET(JulianDayFun), DUCKDB_AGGREGATE_FUNCTION(KahanSumFun), DUCKDB_AGGREGATE_FUNCTION(KurtosisFun), + DUCKDB_AGGREGATE_FUNCTION(KurtosisPopFun), DUCKDB_SCALAR_FUNCTION_SET(LastDayFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(LcmFun), DUCKDB_SCALAR_FUNCTION_SET(LeastFun), @@ -218,6 +224,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_SET(ListGradeUpFun), DUCKDB_SCALAR_FUNCTION_SET(ListInnerProductFun), DUCKDB_SCALAR_FUNCTION_ALIAS(ListPackFun), + DUCKDB_SCALAR_FUNCTION(ListReduceFun), DUCKDB_SCALAR_FUNCTION_SET(ListReverseSortFun), DUCKDB_SCALAR_FUNCTION_SET(ListSliceFun), DUCKDB_SCALAR_FUNCTION_SET(ListSortFun), @@ -263,6 +270,10 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_SET(NextAfterFun), DUCKDB_SCALAR_FUNCTION_ALIAS(NowFun), DUCKDB_SCALAR_FUNCTION_ALIAS(OrdFun), + DUCKDB_SCALAR_FUNCTION_SET(ParseDirnameFun), + DUCKDB_SCALAR_FUNCTION_SET(ParseDirpathFun), + DUCKDB_SCALAR_FUNCTION_SET(ParseFilenameFun), + DUCKDB_SCALAR_FUNCTION_SET(ParsePathFun), DUCKDB_SCALAR_FUNCTION(PiFun), DUCKDB_SCALAR_FUNCTION_ALIAS(PositionFun), DUCKDB_SCALAR_FUNCTION_ALIAS(PowFun), @@ -276,6 +287,8 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION(RadiansFun), DUCKDB_SCALAR_FUNCTION(RandomFun), DUCKDB_SCALAR_FUNCTION_SET(ListRangeFun), + DUCKDB_SCALAR_FUNCTION_ALIAS(ReduceFun), + DUCKDB_SCALAR_FUNCTION(RegexpEscapeFun), DUCKDB_SCALAR_FUNCTION_SET_ALIAS(RegexpSplitToArrayFun), DUCKDB_AGGREGATE_FUNCTION(RegrAvgxFun), DUCKDB_AGGREGATE_FUNCTION(RegrAvgyFun), @@ -371,6 +384,7 @@ static StaticFunctionDefinition internal_functions[] = { DUCKDB_SCALAR_FUNCTION_SET(WeekFun), DUCKDB_SCALAR_FUNCTION_SET(WeekDayFun), DUCKDB_SCALAR_FUNCTION_SET(WeekOfYearFun), + DUCKDB_SCALAR_FUNCTION(WhichSecretFun), DUCKDB_SCALAR_FUNCTION_SET(BitwiseXorFun), DUCKDB_SCALAR_FUNCTION_SET(YearFun), DUCKDB_SCALAR_FUNCTION_SET(YearWeekFun), diff --git a/src/duckdb/src/core_functions/lambda_functions.cpp b/src/duckdb/src/core_functions/lambda_functions.cpp index 374e254ea..3b67f8809 100644 --- a/src/duckdb/src/core_functions/lambda_functions.cpp +++ b/src/duckdb/src/core_functions/lambda_functions.cpp @@ -12,19 +12,6 @@ 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 { @@ -173,9 +160,9 @@ struct ListFilterFunctor { } }; -vector GetColumnInfo(DataChunk &args, const idx_t row_count) { +vector LambdaFunctions::GetColumnInfo(DataChunk &args, const idx_t row_count) { - vector data; + 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]); @@ -184,9 +171,10 @@ vector GetColumnInfo(DataChunk &args, const idx_t row_count) { return data; } -vector> GetInconstantColumnInfo(vector &data) { +vector> +LambdaFunctions::GetInconstantColumnInfo(vector &data) { - vector> inconstant_info; + vector> inconstant_info; for (auto &entry : data) { if (entry.vector.get().GetVectorType() != VectorType::CONSTANT_VECTOR) { inconstant_info.push_back(entry); @@ -195,8 +183,8 @@ vector> GetInconstantColumnInfo(vector &column_infos, const Vector &index_vector, +void ExecuteExpression(const idx_t elem_cnt, const LambdaFunctions::ColumnInfo &column_info, + const vector &column_infos, const Vector &index_vector, LambdaExecuteInfo &info) { info.input_chunk.SetCardinality(elem_cnt); @@ -219,7 +207,7 @@ void ExecuteExpression(const idx_t elem_cnt, const LambdaColumnInfo &column_info for (idx_t i = 0; i < column_infos.size(); i++) { if (column_infos[i].vector.get().GetVectorType() == VectorType::CONSTANT_VECTOR) { - // only reference constant vectors + // only reference constant vectorsl info.input_chunk.data[i + slice_offset].Reference(column_infos[i].vector); } else { @@ -279,48 +267,42 @@ LogicalType LambdaFunctions::BindBinaryLambda(const idx_t parameter_idx, const L } } +LogicalType LambdaFunctions::BindTernaryLambda(const idx_t parameter_idx, const LogicalType &list_child_type) { + switch (parameter_idx) { + case 0: + return list_child_type; + case 1: + return list_child_type; + case 2: + return LogicalType::BIGINT; + default: + throw BinderException("This lambda function only supports up to three 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); + bool result_is_null = false; + LambdaFunctions::LambdaInfo info(args, state, result, result_is_null); + if (result_is_null) { 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); + auto result_entries = FlatVector::GetData(result); + auto inconstant_column_infos = LambdaFunctions::GetInconstantColumnInfo(info.column_infos); // 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); + auto child_vector_size = ListVector::GetListSize(args.data[0]); + LambdaFunctions::ColumnInfo child_info(*info.child_vector); + info.child_vector->ToUnifiedFormat(child_vector_size, child_info.format); // get the expression executor - LambdaExecuteInfo execute_info(state.GetContext(), *lambda_expr, args, bind_info.has_index, child_vector); + LambdaExecuteInfo execute_info(state.GetContext(), *info.lambda_expr, args, info.has_index, *info.child_vector); // get list_filter specific info ListFilterInfo list_filter_info; - FUNCTION_FUNCTOR::ReserveNewLengths(list_filter_info.entry_lengths, row_count); + FUNCTION_FUNCTOR::ReserveNewLengths(list_filter_info.entry_lengths, info.row_count); // additional index vector Vector index_vector(LogicalType::BIGINT); @@ -328,14 +310,14 @@ void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { // 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++) { + for (idx_t row_idx = 0; row_idx < info.row_count; row_idx++) { - auto list_idx = list_column_format.sel->get_index(row_idx); - const auto &list_entry = list_entries[list_idx]; + auto list_idx = info.list_column_format.sel->get_index(row_idx); + const auto &list_entry = info.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); + if (!info.list_column_format.validity.RowIsValid(list_idx)) { + info.result_validity->SetInvalid(row_idx); FUNCTION_FUNCTOR::PushEmptyList(list_filter_info.entry_lengths); continue; } @@ -354,7 +336,7 @@ void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { if (elem_cnt == STANDARD_VECTOR_SIZE) { execute_info.lambda_chunk.Reset(); - ExecuteExpression(elem_cnt, child_info, column_infos, index_vector, execute_info); + ExecuteExpression(elem_cnt, child_info, 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, @@ -362,6 +344,7 @@ void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { elem_cnt = 0; } + // FIXME: reuse same selection vector for inconstant rows // adjust indexes for slicing child_info.sel.set_index(elem_cnt, list_entry.offset + child_idx); for (auto &entry : inconstant_column_infos) { @@ -369,7 +352,7 @@ void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { } // set the index vector - if (bind_info.has_index) { + if (info.has_index) { index_vector.SetValue(elem_cnt, Value::BIGINT(child_idx + 1)); } @@ -378,20 +361,19 @@ void ExecuteLambda(DataChunk &args, ExpressionState &state, Vector &result) { } execute_info.lambda_chunk.Reset(); - ExecuteExpression(elem_cnt, child_info, column_infos, index_vector, execute_info); + ExecuteExpression(elem_cnt, child_info, 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) { + if (info.is_all_constant && !info.is_volatile) { result.SetVectorType(VectorType::CONSTANT_VECTOR); } } -unique_ptr LambdaFunctions::ListLambdaBind(ClientContext &context, ScalarFunction &bound_function, - vector> &arguments, - const bool has_index) { - +unique_ptr LambdaFunctions::ListLambdaPrepareBind(vector> &arguments, + ClientContext &context, + ScalarFunction &bound_function) { // NULL list parameter if (arguments[0]->return_type.id() == LogicalTypeId::SQLNULL) { bound_function.arguments[0] = LogicalType::SQLNULL; @@ -405,10 +387,21 @@ unique_ptr LambdaFunctions::ListLambdaBind(ClientContext &context, arguments[0] = BoundCastExpression::AddArrayCastToList(context, std::move(arguments[0])); D_ASSERT(arguments[0]->return_type.id() == LogicalTypeId::LIST); + return nullptr; +} + +unique_ptr LambdaFunctions::ListLambdaBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments, + const bool has_index) { + unique_ptr bind_data = ListLambdaPrepareBind(arguments, context, bound_function); + if (bind_data) { + return bind_data; + } // 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); } diff --git a/src/duckdb/src/core_functions/scalar/array/array_functions.cpp b/src/duckdb/src/core_functions/scalar/array/array_functions.cpp index ac1f7664e..9840ba1cc 100644 --- a/src/duckdb/src/core_functions/scalar/array/array_functions.cpp +++ b/src/duckdb/src/core_functions/scalar/array/array_functions.cpp @@ -216,7 +216,7 @@ static unique_ptr ArrayGenericBinaryBind(ClientContext &context, S auto size = left_size; auto child_type = - LogicalType::MaxLogicalType(ArrayType::GetChildType(left_type), ArrayType::GetChildType(right_type)); + LogicalType::MaxLogicalType(context, 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)); diff --git a/src/duckdb/src/core_functions/scalar/array/array_value.cpp b/src/duckdb/src/core_functions/scalar/array/array_value.cpp index d9325e745..ac4f0bd24 100644 --- a/src/duckdb/src/core_functions/scalar/array/array_value.cpp +++ b/src/duckdb/src/core_functions/scalar/array/array_value.cpp @@ -51,7 +51,7 @@ static unique_ptr ArrayValueBind(ClientContext &context, ScalarFun // 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); + child_type = LogicalType::MaxLogicalType(context, child_type, arguments[i]->return_type); } if (arguments.size() > ArrayType::MAX_ARRAY_SIZE) { diff --git a/src/duckdb/src/core_functions/scalar/blob/create_sort_key.cpp b/src/duckdb/src/core_functions/scalar/blob/create_sort_key.cpp new file mode 100644 index 000000000..880acd2c8 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/blob/create_sort_key.cpp @@ -0,0 +1,686 @@ +#include "duckdb/core_functions/scalar/blob_functions.hpp" +#include "duckdb/execution/expression_executor.hpp" +#include "duckdb/common/enums/order_type.hpp" +#include "duckdb/common/radix.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression_binder.hpp" + +namespace duckdb { + +struct OrderModifiers { + OrderModifiers(OrderType order_type, OrderByNullType null_type) : order_type(order_type), null_type(null_type) { + } + + OrderType order_type; + OrderByNullType null_type; + + bool operator==(const OrderModifiers &other) const { + return order_type == other.order_type && null_type == other.null_type; + } + + static OrderModifiers Parse(const string &val) { + auto lcase = StringUtil::Replace(StringUtil::Lower(val), "_", " "); + OrderType order_type; + if (StringUtil::StartsWith(lcase, "asc")) { + order_type = OrderType::ASCENDING; + } else if (StringUtil::StartsWith(lcase, "desc")) { + order_type = OrderType::DESCENDING; + } else { + throw BinderException("create_sort_key modifier must start with either ASC or DESC"); + } + OrderByNullType null_type; + if (StringUtil::EndsWith(lcase, "nulls first")) { + null_type = OrderByNullType::NULLS_FIRST; + } else if (StringUtil::EndsWith(lcase, "nulls last")) { + null_type = OrderByNullType::NULLS_LAST; + } else { + throw BinderException("create_sort_key modifier must end with either NULLS FIRST or NULLS LAST"); + } + return OrderModifiers(order_type, null_type); + } +}; + +struct CreateSortKeyBindData : public FunctionData { + vector modifiers; + + bool Equals(const FunctionData &other_p) const override { + auto &other = other_p.Cast(); + return modifiers == other.modifiers; + } + unique_ptr Copy() const override { + auto result = make_uniq(); + result->modifiers = modifiers; + return std::move(result); + } +}; + +unique_ptr CreateSortKeyBind(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + if (arguments.size() % 2 != 0) { + throw BinderException( + "Arguments to create_sort_key must be [key1, sort_specifier1, key2, sort_specifier2, ...]"); + } + auto result = make_uniq(); + for (idx_t i = 1; i < arguments.size(); i += 2) { + if (!arguments[i]->IsFoldable()) { + throw BinderException("sort_specifier must be a constant value - but got %s", arguments[i]->ToString()); + } + + // Rebind to return a date if we are truncating that far + Value sort_specifier = ExpressionExecutor::EvaluateScalar(context, *arguments[i]); + if (sort_specifier.IsNull()) { + throw BinderException("sort_specifier cannot be NULL"); + } + auto sort_specifier_str = sort_specifier.ToString(); + result->modifiers.push_back(OrderModifiers::Parse(sort_specifier_str)); + } + // push collations + for (idx_t i = 0; i < arguments.size(); i += 2) { + ExpressionBinder::PushCollation(context, arguments[i], arguments[i]->return_type, false); + } + // check if all types are constant + bool all_constant = true; + idx_t constant_size = 0; + for (idx_t i = 0; i < arguments.size(); i += 2) { + auto physical_type = arguments[i]->return_type.InternalType(); + if (!TypeIsConstantSize(physical_type)) { + all_constant = false; + } else { + // we always add one byte for the validity + constant_size += GetTypeIdSize(physical_type) + 1; + } + } + if (all_constant) { + if (constant_size <= sizeof(int64_t)) { + bound_function.return_type = LogicalType::BIGINT; + } + } + return std::move(result); +} + +//===--------------------------------------------------------------------===// +// Operators +//===--------------------------------------------------------------------===// +struct SortKeyVectorData { + static constexpr data_t NULL_FIRST_BYTE = 1; + static constexpr data_t NULL_LAST_BYTE = 2; + static constexpr data_t STRING_DELIMITER = 0; + static constexpr data_t LIST_DELIMITER = 0; + static constexpr data_t BLOB_ESCAPE_CHARACTER = 1; + + SortKeyVectorData(Vector &input, idx_t size, OrderModifiers modifiers) : vec(input) { + input.ToUnifiedFormat(size, format); + this->size = size; + + null_byte = NULL_FIRST_BYTE; + valid_byte = NULL_LAST_BYTE; + if (modifiers.null_type == OrderByNullType::NULLS_LAST) { + std::swap(null_byte, valid_byte); + } + + // NULLS FIRST/NULLS LAST passed in by the user are only respected at the top level + // within nested types NULLS LAST/NULLS FIRST is dependent on ASC/DESC order instead + // don't blame me this is what Postgres does + auto child_null_type = + modifiers.order_type == OrderType::ASCENDING ? OrderByNullType::NULLS_LAST : OrderByNullType::NULLS_FIRST; + OrderModifiers child_modifiers(modifiers.order_type, child_null_type); + switch (input.GetType().InternalType()) { + case PhysicalType::STRUCT: { + auto &children = StructVector::GetEntries(input); + for (auto &child : children) { + child_data.push_back(make_uniq(*child, size, child_modifiers)); + } + break; + } + case PhysicalType::ARRAY: { + auto &child_entry = ArrayVector::GetEntry(input); + auto array_size = ArrayType::GetSize(input.GetType()); + child_data.push_back(make_uniq(child_entry, size * array_size, child_modifiers)); + break; + } + case PhysicalType::LIST: { + auto &child_entry = ListVector::GetEntry(input); + auto child_size = ListVector::GetListSize(input); + child_data.push_back(make_uniq(child_entry, child_size, child_modifiers)); + break; + } + default: + break; + } + } + // disable copy constructors + SortKeyVectorData(const SortKeyVectorData &other) = delete; + SortKeyVectorData &operator=(const SortKeyVectorData &) = delete; + + PhysicalType GetPhysicalType() { + return vec.GetType().InternalType(); + } + + Vector &vec; + idx_t size; + UnifiedVectorFormat format; + vector> child_data; + data_t null_byte; + data_t valid_byte; +}; + +template +struct SortKeyConstantOperator { + using TYPE = T; + + static idx_t GetEncodeLength(TYPE input) { + return sizeof(T); + } + + static idx_t Encode(data_ptr_t result, TYPE input) { + Radix::EncodeData(result, input); + return sizeof(T); + } +}; + +struct SortKeyVarcharOperator { + using TYPE = string_t; + + static idx_t GetEncodeLength(TYPE input) { + return input.GetSize() + 1; + } + + static idx_t Encode(data_ptr_t result, TYPE input) { + auto input_data = input.GetDataUnsafe(); + auto input_size = input.GetSize(); + for (idx_t r = 0; r < input_size; r++) { + result[r] = input_data[r] + 1; + } + result[input_size] = SortKeyVectorData::STRING_DELIMITER; // null-byte delimiter + return input_size + 1; + } +}; + +struct SortKeyBlobOperator { + using TYPE = string_t; + + static idx_t GetEncodeLength(TYPE input) { + auto input_data = data_ptr_t(input.GetDataUnsafe()); + auto input_size = input.GetSize(); + idx_t escaped_characters = 0; + for (idx_t r = 0; r < input_size; r++) { + if (input_data[r] <= 1) { + // we escape both \x00 and \x01 + escaped_characters++; + } + } + return input.GetSize() + escaped_characters + 1; + } + + static idx_t Encode(data_ptr_t result, TYPE input) { + auto input_data = data_ptr_t(input.GetDataUnsafe()); + auto input_size = input.GetSize(); + idx_t result_offset = 0; + for (idx_t r = 0; r < input_size; r++) { + if (input_data[r] <= 1) { + // we escape both \x00 and \x01 with \x01 + result[result_offset++] = SortKeyVectorData::BLOB_ESCAPE_CHARACTER; + result[result_offset++] = input_data[r]; + } else { + result[result_offset++] = input_data[r]; + } + } + result[result_offset++] = SortKeyVectorData::STRING_DELIMITER; // null-byte delimiter + return result_offset; + } +}; + +struct SortKeyListEntry { + static bool IsArray() { + return false; + } + + static list_entry_t GetListEntry(SortKeyVectorData &vector_data, idx_t idx) { + auto data = UnifiedVectorFormat::GetData(vector_data.format); + return data[idx]; + } +}; + +struct SortKeyArrayEntry { + static bool IsArray() { + return true; + } + + static list_entry_t GetListEntry(SortKeyVectorData &vector_data, idx_t idx) { + auto array_size = ArrayType::GetSize(vector_data.vec.GetType()); + return list_entry_t(array_size * idx, array_size); + } +}; + +struct SortKeyChunk { + SortKeyChunk(idx_t start, idx_t end) : start(start), end(end), has_result_index(false) { + } + SortKeyChunk(idx_t start, idx_t end, idx_t result_index) + : start(start), end(end), result_index(result_index), has_result_index(true) { + } + + idx_t start; + idx_t end; + idx_t result_index; + bool has_result_index; + + inline idx_t GetResultIndex(idx_t r) { + return has_result_index ? result_index : r; + } +}; + +//===--------------------------------------------------------------------===// +// Get Sort Key Length +//===--------------------------------------------------------------------===// +struct SortKeyLengthInfo { + explicit SortKeyLengthInfo(idx_t size) : constant_length(0) { + variable_lengths.resize(size, 0); + } + + idx_t constant_length; + unsafe_vector variable_lengths; +}; + +static void GetSortKeyLengthRecursive(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyLengthInfo &result); + +template +void TemplatedGetSortKeyLength(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyLengthInfo &result) { + auto &format = vector_data.format; + auto data = UnifiedVectorFormat::GetData(vector_data.format); + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto idx = format.sel->get_index(r); + auto result_index = chunk.GetResultIndex(r); + result.variable_lengths[result_index]++; // every value is prefixed by a validity byte + + if (!format.validity.RowIsValid(idx)) { + continue; + } + result.variable_lengths[result_index] += OP::GetEncodeLength(data[idx]); + } +} + +void GetSortKeyLengthStruct(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyLengthInfo &result) { + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto result_index = chunk.GetResultIndex(r); + result.variable_lengths[result_index]++; // every struct is prefixed by a validity byte + } + // now recursively call GetSortKeyLength on the child elements + for (auto &child_data : vector_data.child_data) { + GetSortKeyLengthRecursive(*child_data, chunk, result); + } +} + +template +void GetSortKeyLengthList(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyLengthInfo &result) { + auto &child_data = vector_data.child_data[0]; + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto idx = vector_data.format.sel->get_index(r); + auto result_index = chunk.GetResultIndex(r); + result.variable_lengths[result_index]++; // every list is prefixed by a validity byte + + if (!vector_data.format.validity.RowIsValid(idx)) { + if (!OP::IsArray()) { + // for arrays we need to fill in the child vector for all elements, even if the top-level array is NULL + continue; + } + } + auto list_entry = OP::GetListEntry(vector_data, idx); + // for each non-null list we have an "end of list" delimiter + result.variable_lengths[result_index]++; + if (list_entry.length > 0) { + // recursively call GetSortKeyLength for the children of this list + SortKeyChunk child_chunk(list_entry.offset, list_entry.offset + list_entry.length, result_index); + GetSortKeyLengthRecursive(*child_data, child_chunk, result); + } + } +} + +static void GetSortKeyLengthRecursive(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyLengthInfo &result) { + auto physical_type = vector_data.GetPhysicalType(); + // handle variable lengths + switch (physical_type) { + case PhysicalType::BOOL: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::UINT8: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INT8: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::UINT16: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INT16: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::UINT32: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INT32: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::UINT64: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INT64: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::FLOAT: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::DOUBLE: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INTERVAL: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::UINT128: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::INT128: + TemplatedGetSortKeyLength>(vector_data, chunk, result); + break; + case PhysicalType::VARCHAR: + if (vector_data.vec.GetType().id() == LogicalTypeId::VARCHAR) { + TemplatedGetSortKeyLength(vector_data, chunk, result); + } else { + TemplatedGetSortKeyLength(vector_data, chunk, result); + } + break; + case PhysicalType::STRUCT: + GetSortKeyLengthStruct(vector_data, chunk, result); + break; + case PhysicalType::LIST: + GetSortKeyLengthList(vector_data, chunk, result); + break; + case PhysicalType::ARRAY: + GetSortKeyLengthList(vector_data, chunk, result); + break; + default: + throw NotImplementedException("Unsupported physical type %s in GetSortKeyLength", physical_type); + } +} + +static void GetSortKeyLength(SortKeyVectorData &vector_data, SortKeyLengthInfo &result) { + // top-level method + auto physical_type = vector_data.GetPhysicalType(); + if (TypeIsConstantSize(physical_type)) { + // every row is prefixed by a validity byte + result.constant_length += 1; + result.constant_length += GetTypeIdSize(physical_type); + return; + } + GetSortKeyLengthRecursive(vector_data, SortKeyChunk(0, vector_data.size), result); +} + +//===--------------------------------------------------------------------===// +// Construct Sort Key +//===--------------------------------------------------------------------===// +struct SortKeyConstructInfo { + SortKeyConstructInfo(OrderModifiers modifiers_p, unsafe_vector &offsets, data_ptr_t *result_data) + : modifiers(modifiers_p), offsets(offsets), result_data(result_data) { + flip_bytes = modifiers.order_type == OrderType::DESCENDING; + } + + OrderModifiers modifiers; + unsafe_vector &offsets; + data_ptr_t *result_data; + bool flip_bytes; +}; + +static void ConstructSortKeyRecursive(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyConstructInfo &info); + +template +void TemplatedConstructSortKey(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyConstructInfo &info) { + auto data = UnifiedVectorFormat::GetData(vector_data.format); + auto &offsets = info.offsets; + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto result_index = chunk.GetResultIndex(r); + auto idx = vector_data.format.sel->get_index(r); + auto &offset = offsets[result_index]; + auto result_ptr = info.result_data[result_index]; + if (!vector_data.format.validity.RowIsValid(idx)) { + // NULL value - write the null byte and skip + result_ptr[offset++] = vector_data.null_byte; + continue; + } + // valid value - write the validity byte + result_ptr[offset++] = vector_data.valid_byte; + idx_t encode_len = OP::Encode(result_ptr + offset, data[idx]); + if (info.flip_bytes) { + // descending order - so flip bytes + for (idx_t b = offset; b < offset + encode_len; b++) { + result_ptr[b] = ~result_ptr[b]; + } + } + offset += encode_len; + } +} + +void ConstructSortKeyStruct(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyConstructInfo &info) { + bool list_of_structs = chunk.has_result_index; + // write the validity data of the struct + auto &offsets = info.offsets; + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto result_index = chunk.GetResultIndex(r); + auto idx = vector_data.format.sel->get_index(r); + auto &offset = offsets[result_index]; + auto result_ptr = info.result_data[result_index]; + if (!vector_data.format.validity.RowIsValid(idx)) { + // NULL value - write the null byte and skip + result_ptr[offset++] = vector_data.null_byte; + } else { + // valid value - write the validity byte + result_ptr[offset++] = vector_data.valid_byte; + } + if (list_of_structs) { + // for a list of structs we need to write the child data for every iteration + // since the final layout needs to be + // [struct1][struct2][...] + for (auto &child : vector_data.child_data) { + SortKeyChunk child_chunk(r, r + 1, result_index); + ConstructSortKeyRecursive(*child, child_chunk, info); + } + } + } + if (!list_of_structs) { + for (auto &child : vector_data.child_data) { + ConstructSortKeyRecursive(*child, chunk, info); + } + } +} + +template +void ConstructSortKeyList(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyConstructInfo &info) { + auto &offsets = info.offsets; + for (idx_t r = chunk.start; r < chunk.end; r++) { + auto result_index = chunk.GetResultIndex(r); + auto idx = vector_data.format.sel->get_index(r); + auto &offset = offsets[result_index]; + auto result_ptr = info.result_data[result_index]; + if (!vector_data.format.validity.RowIsValid(idx)) { + // NULL value - write the null byte and skip + result_ptr[offset++] = vector_data.null_byte; + if (!OP::IsArray()) { + // for arrays we always write the child elements - also if the top-level array is NULL + continue; + } + } else { + // valid value - write the validity byte + result_ptr[offset++] = vector_data.valid_byte; + } + + auto list_entry = OP::GetListEntry(vector_data, idx); + // recurse and write the list elements + if (list_entry.length > 0) { + SortKeyChunk child_chunk(list_entry.offset, list_entry.offset + list_entry.length, result_index); + ConstructSortKeyRecursive(*vector_data.child_data[0], child_chunk, info); + } + + // write the end-of-list delimiter + result_ptr[offset++] = info.flip_bytes ? ~SortKeyVectorData::LIST_DELIMITER : SortKeyVectorData::LIST_DELIMITER; + } +} + +static void ConstructSortKeyRecursive(SortKeyVectorData &vector_data, SortKeyChunk chunk, SortKeyConstructInfo &info) { + switch (vector_data.GetPhysicalType()) { + case PhysicalType::BOOL: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::UINT8: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INT8: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::UINT16: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INT16: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::UINT32: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INT32: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::UINT64: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INT64: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::FLOAT: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::DOUBLE: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INTERVAL: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::UINT128: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::INT128: + TemplatedConstructSortKey>(vector_data, chunk, info); + break; + case PhysicalType::VARCHAR: + if (vector_data.vec.GetType().id() == LogicalTypeId::VARCHAR) { + TemplatedConstructSortKey(vector_data, chunk, info); + } else { + TemplatedConstructSortKey(vector_data, chunk, info); + } + break; + case PhysicalType::STRUCT: + ConstructSortKeyStruct(vector_data, chunk, info); + break; + case PhysicalType::LIST: + ConstructSortKeyList(vector_data, chunk, info); + break; + case PhysicalType::ARRAY: + ConstructSortKeyList(vector_data, chunk, info); + break; + default: + throw NotImplementedException("Unsupported type %s in ConstructSortKey", vector_data.vec.GetType()); + } +} + +static void ConstructSortKey(SortKeyVectorData &vector_data, SortKeyConstructInfo &info) { + ConstructSortKeyRecursive(vector_data, SortKeyChunk(0, vector_data.size), info); +} + +static void PrepareSortData(Vector &result, idx_t size, SortKeyLengthInfo &key_lengths, data_ptr_t *data_pointers) { + switch (result.GetType().id()) { + case LogicalTypeId::BLOB: { + auto result_data = FlatVector::GetData(result); + for (idx_t r = 0; r < size; r++) { + auto blob_size = key_lengths.variable_lengths[r] + key_lengths.constant_length; + result_data[r] = StringVector::EmptyString(result, blob_size); + data_pointers[r] = data_ptr_cast(result_data[r].GetDataWriteable()); +#ifdef DEBUG + memset(data_pointers[r], 0xFF, blob_size); +#endif + } + break; + } + case LogicalTypeId::BIGINT: { + auto result_data = FlatVector::GetData(result); + for (idx_t r = 0; r < size; r++) { + result_data[r] = 0; + data_pointers[r] = data_ptr_cast(&result_data[r]); + } + break; + } + default: + throw InternalException("Unsupported key type for CreateSortKey"); + } +} + +static void FinalizeSortData(Vector &result, idx_t size) { + switch (result.GetType().id()) { + case LogicalTypeId::BLOB: { + auto result_data = FlatVector::GetData(result); + // call Finalize on the result + for (idx_t r = 0; r < size; r++) { + result_data[r].Finalize(); + } + break; + } + case LogicalTypeId::BIGINT: { + auto result_data = FlatVector::GetData(result); + for (idx_t r = 0; r < size; r++) { + result_data[r] = BSwap(result_data[r]); + } + break; + } + default: + throw InternalException("Unsupported key type for CreateSortKey"); + } +} + +static void CreateSortKeyFunction(DataChunk &args, ExpressionState &state, Vector &result) { + auto &bind_data = state.expr.Cast().bind_info->Cast(); + + // prepare the sort key data + vector> sort_key_data; + for (idx_t c = 0; c < args.ColumnCount(); c += 2) { + sort_key_data.push_back(make_uniq(args.data[c], args.size(), bind_data.modifiers[c / 2])); + } + + // two phases + // a) get the length of the final sorted key + // b) allocate the sorted key and construct + // we do all of this in a vectorized manner + SortKeyLengthInfo key_lengths(args.size()); + for (auto &vector_data : sort_key_data) { + GetSortKeyLength(*vector_data, key_lengths); + } + // allocate the empty sort keys + auto data_pointers = unique_ptr(new data_ptr_t[args.size()]); + PrepareSortData(result, args.size(), key_lengths, data_pointers.get()); + + unsafe_vector offsets; + offsets.resize(args.size(), 0); + // now construct the sort keys + for (idx_t c = 0; c < sort_key_data.size(); c++) { + SortKeyConstructInfo info(bind_data.modifiers[c], offsets, data_pointers.get()); + ConstructSortKey(*sort_key_data[c], info); + } + FinalizeSortData(result, args.size()); + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +ScalarFunction CreateSortKeyFun::GetFunction() { + ScalarFunction sort_key_function("create_sort_key", {LogicalType::ANY}, LogicalType::BLOB, CreateSortKeyFunction, + CreateSortKeyBind); + sort_key_function.varargs = LogicalType::ANY; + sort_key_function.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + return sort_key_function; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/blob/encode.cpp b/src/duckdb/src/core_functions/scalar/blob/encode.cpp index 75e66cc96..ff11f2f6c 100644 --- a/src/duckdb/src/core_functions/scalar/blob/encode.cpp +++ b/src/duckdb/src/core_functions/scalar/blob/encode.cpp @@ -1,5 +1,6 @@ #include "duckdb/core_functions/scalar/blob_functions.hpp" #include "utf8proc_wrapper.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/core_functions/scalar/date/current.cpp b/src/duckdb/src/core_functions/scalar/date/current.cpp index e8b069e6d..159575315 100644 --- a/src/duckdb/src/core_functions/scalar/date/current.cpp +++ b/src/duckdb/src/core_functions/scalar/date/current.cpp @@ -35,19 +35,19 @@ static void CurrentTimestampFunction(DataChunk &input, ExpressionState &state, V ScalarFunction CurrentTimeFun::GetFunction() { ScalarFunction current_time({}, LogicalType::TIME, CurrentTimeFunction); - current_time.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + current_time.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; return current_time; } ScalarFunction CurrentDateFun::GetFunction() { ScalarFunction current_date({}, LogicalType::DATE, CurrentDateFunction); - current_date.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + current_date.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; return current_date; } ScalarFunction GetCurrentTimestampFun::GetFunction() { ScalarFunction current_timestamp({}, LogicalType::TIMESTAMP_TZ, CurrentTimestampFunction); - current_timestamp.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + current_timestamp.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; return current_timestamp; } diff --git a/src/duckdb/src/core_functions/scalar/date/date_part.cpp b/src/duckdb/src/core_functions/scalar/date/date_part.cpp index 019d8c5ea..4ffde4e73 100644 --- a/src/duckdb/src/core_functions/scalar/date/date_part.cpp +++ b/src/duckdb/src/core_functions/scalar/date/date_part.cpp @@ -5,6 +5,7 @@ #include "duckdb/common/string_util.hpp" #include "duckdb/common/enum_util.hpp" #include "duckdb/common/types/date.hpp" +#include "duckdb/common/types/time.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" @@ -43,6 +44,7 @@ DatePartSpecifier GetDateTypePartSpecifier(const string &specifier, LogicalType } break; case LogicalType::TIME: + case LogicalType::TIME_TZ: switch (part) { case DatePartSpecifier::MICROSECONDS: case DatePartSpecifier::MILLISECONDS: @@ -358,7 +360,7 @@ struct DatePart { struct EpochNanosecondsOperator { template static inline TR Operation(TA input) { - return input.micros * Interval::NANOS_PER_MICRO; + return Timestamp::GetEpochNanoSeconds(input); } template @@ -370,7 +372,7 @@ struct DatePart { struct EpochMicrosecondsOperator { template static inline TR Operation(TA input) { - return input.micros; + return Timestamp::GetEpochMicroSeconds(input); } template @@ -382,7 +384,7 @@ struct DatePart { struct EpochMillisOperator { template static inline TR Operation(TA input) { - return input.micros / Interval::MICROS_PER_MSEC; + return Timestamp::GetEpochMs(input); } template @@ -494,6 +496,51 @@ struct DatePart { return 0; } + template + static TR Operation(TA interval, TB timetz) { + auto time = Time::NormalizeTimeTZ(timetz); + date_t date(0); + time = Interval::Add(time, interval, date); + auto offset = interval.micros / Interval::MICROS_PER_SEC; + return TR(time, offset); + } + + template + static void BinaryFunction(DataChunk &input, ExpressionState &state, Vector &result) { + D_ASSERT(input.ColumnCount() == 2); + auto &offset = input.data[0]; + auto &timetz = input.data[1]; + + auto func = DatePart::TimezoneOperator::Operation; + BinaryExecutor::Execute(offset, timetz, result, input.size(), func); + } + + template + static unique_ptr PropagateStatistics(ClientContext &context, FunctionStatisticsInput &input) { + return PropagateSimpleDatePartStatistics<0, 0>(input.child_stats); + } + }; + + struct TimezoneHourOperator { + template + static inline TR Operation(TA input) { + // Regular timestamps are UTC. + return 0; + } + + template + static unique_ptr PropagateStatistics(ClientContext &context, FunctionStatisticsInput &input) { + return PropagateSimpleDatePartStatistics<0, 0>(input.child_stats); + } + }; + + struct TimezoneMinuteOperator { + template + static inline TR Operation(TA input) { + // Regular timestamps are UTC. + return 0; + } + template static unique_ptr PropagateStatistics(ClientContext &context, FunctionStatisticsInput &input) { return PropagateSimpleDatePartStatistics<0, 0>(input.child_stats); @@ -512,10 +559,6 @@ struct DatePart { } }; - // These are all zero and have the same restrictions - using TimezoneHourOperator = TimezoneOperator; - using TimezoneMinuteOperator = TimezoneOperator; - struct StructOperator { using part_codes_t = vector; using part_mask_t = uint64_t; @@ -722,6 +765,11 @@ int64_t DatePart::YearOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"year\" not recognized"); } +template <> +int64_t DatePart::YearOperator::Operation(dtime_tz_t input) { + return YearOperator::Operation(input.time()); +} + template <> int64_t DatePart::MonthOperator::Operation(timestamp_t input) { return MonthOperator::Operation(Timestamp::GetDate(input)); @@ -737,6 +785,11 @@ int64_t DatePart::MonthOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"month\" not recognized"); } +template <> +int64_t DatePart::MonthOperator::Operation(dtime_tz_t input) { + return MonthOperator::Operation(input.time()); +} + template <> int64_t DatePart::DayOperator::Operation(timestamp_t input) { return DayOperator::Operation(Timestamp::GetDate(input)); @@ -752,6 +805,11 @@ int64_t DatePart::DayOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"day\" not recognized"); } +template <> +int64_t DatePart::DayOperator::Operation(dtime_tz_t input) { + return DayOperator::Operation(input.time()); +} + template <> int64_t DatePart::DecadeOperator::Operation(interval_t input) { return input.months / Interval::MONTHS_PER_DECADE; @@ -762,6 +820,11 @@ int64_t DatePart::DecadeOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"decade\" not recognized"); } +template <> +int64_t DatePart::DecadeOperator::Operation(dtime_tz_t input) { + return DecadeOperator::Operation(input.time()); +} + template <> int64_t DatePart::CenturyOperator::Operation(interval_t input) { return input.months / Interval::MONTHS_PER_CENTURY; @@ -772,6 +835,11 @@ int64_t DatePart::CenturyOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"century\" not recognized"); } +template <> +int64_t DatePart::CenturyOperator::Operation(dtime_tz_t input) { + return CenturyOperator::Operation(input.time()); +} + template <> int64_t DatePart::MillenniumOperator::Operation(interval_t input) { return input.months / Interval::MONTHS_PER_MILLENIUM; @@ -782,6 +850,11 @@ int64_t DatePart::MillenniumOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"millennium\" not recognized"); } +template <> +int64_t DatePart::MillenniumOperator::Operation(dtime_tz_t input) { + return MillenniumOperator::Operation(input.time()); +} + template <> int64_t DatePart::QuarterOperator::Operation(timestamp_t input) { return QuarterOperator::Operation(Timestamp::GetDate(input)); @@ -797,6 +870,11 @@ int64_t DatePart::QuarterOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"quarter\" not recognized"); } +template <> +int64_t DatePart::QuarterOperator::Operation(dtime_tz_t input) { + return QuarterOperator::Operation(input.time()); +} + template <> int64_t DatePart::DayOfWeekOperator::Operation(timestamp_t input) { return DayOfWeekOperator::Operation(Timestamp::GetDate(input)); @@ -812,6 +890,11 @@ int64_t DatePart::DayOfWeekOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"dow\" not recognized"); } +template <> +int64_t DatePart::DayOfWeekOperator::Operation(dtime_tz_t input) { + return DayOfWeekOperator::Operation(input.time()); +} + template <> int64_t DatePart::ISODayOfWeekOperator::Operation(timestamp_t input) { return ISODayOfWeekOperator::Operation(Timestamp::GetDate(input)); @@ -827,6 +910,11 @@ int64_t DatePart::ISODayOfWeekOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"isodow\" not recognized"); } +template <> +int64_t DatePart::ISODayOfWeekOperator::Operation(dtime_tz_t input) { + return ISODayOfWeekOperator::Operation(input.time()); +} + template <> int64_t DatePart::DayOfYearOperator::Operation(timestamp_t input) { return DayOfYearOperator::Operation(Timestamp::GetDate(input)); @@ -842,6 +930,11 @@ int64_t DatePart::DayOfYearOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"doy\" not recognized"); } +template <> +int64_t DatePart::DayOfYearOperator::Operation(dtime_tz_t input) { + return DayOfYearOperator::Operation(input.time()); +} + template <> int64_t DatePart::WeekOperator::Operation(timestamp_t input) { return WeekOperator::Operation(Timestamp::GetDate(input)); @@ -857,6 +950,11 @@ int64_t DatePart::WeekOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"week\" not recognized"); } +template <> +int64_t DatePart::WeekOperator::Operation(dtime_tz_t input) { + return WeekOperator::Operation(input.time()); +} + template <> int64_t DatePart::ISOYearOperator::Operation(timestamp_t input) { return ISOYearOperator::Operation(Timestamp::GetDate(input)); @@ -872,6 +970,11 @@ int64_t DatePart::ISOYearOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"isoyear\" not recognized"); } +template <> +int64_t DatePart::ISOYearOperator::Operation(dtime_tz_t input) { + return ISOYearOperator::Operation(input.time()); +} + template <> int64_t DatePart::YearWeekOperator::Operation(timestamp_t input) { return YearWeekOperator::Operation(Timestamp::GetDate(input)); @@ -889,6 +992,11 @@ int64_t DatePart::YearWeekOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"yearweek\" not recognized"); } +template <> +int64_t DatePart::YearWeekOperator::Operation(dtime_tz_t input) { + return YearWeekOperator::Operation(input.time()); +} + template <> int64_t DatePart::EpochNanosecondsOperator::Operation(timestamp_t input) { return Timestamp::GetEpochNanoSeconds(input); @@ -905,8 +1013,13 @@ int64_t DatePart::EpochNanosecondsOperator::Operation(interval_t input) { } template <> -int64_t DatePart::EpochMicrosecondsOperator::Operation(timestamp_t input) { - return Timestamp::GetEpochMicroSeconds(input); +int64_t DatePart::EpochNanosecondsOperator::Operation(dtime_t input) { + return input.micros * Interval::NANOS_PER_MICRO; +} + +template <> +int64_t DatePart::EpochNanosecondsOperator::Operation(dtime_tz_t input) { + return DatePart::EpochNanosecondsOperator::Operation(input.time()); } template <> @@ -920,8 +1033,13 @@ int64_t DatePart::EpochMicrosecondsOperator::Operation(interval_t input) { } template <> -int64_t DatePart::EpochMillisOperator::Operation(timestamp_t input) { - return Timestamp::GetEpochMs(input); +int64_t DatePart::EpochMicrosecondsOperator::Operation(dtime_t input) { + return input.micros; +} + +template <> +int64_t DatePart::EpochMicrosecondsOperator::Operation(dtime_tz_t input) { + return DatePart::EpochMicrosecondsOperator::Operation(input.time()); } template <> @@ -934,6 +1052,16 @@ int64_t DatePart::EpochMillisOperator::Operation(interval_t input) { return Interval::GetMilli(input); } +template <> +int64_t DatePart::EpochMillisOperator::Operation(dtime_t input) { + return input.micros / Interval::MICROS_PER_MSEC; +} + +template <> +int64_t DatePart::EpochMillisOperator::Operation(dtime_tz_t input) { + return DatePart::EpochMillisOperator::Operation(input.time()); +} + template <> int64_t DatePart::MicrosecondsOperator::Operation(timestamp_t input) { auto time = Timestamp::GetTime(input); @@ -953,6 +1081,11 @@ int64_t DatePart::MicrosecondsOperator::Operation(dtime_t input) { return input.micros % Interval::MICROS_PER_MINUTE; } +template <> +int64_t DatePart::MicrosecondsOperator::Operation(dtime_tz_t input) { + return DatePart::MicrosecondsOperator::Operation(input.time()); +} + template <> int64_t DatePart::MillisecondsOperator::Operation(timestamp_t input) { return MicrosecondsOperator::Operation(input) / Interval::MICROS_PER_MSEC; @@ -968,6 +1101,11 @@ int64_t DatePart::MillisecondsOperator::Operation(dtime_t input) { return MicrosecondsOperator::Operation(input) / Interval::MICROS_PER_MSEC; } +template <> +int64_t DatePart::MillisecondsOperator::Operation(dtime_tz_t input) { + return DatePart::MillisecondsOperator::Operation(input.time()); +} + template <> int64_t DatePart::SecondsOperator::Operation(timestamp_t input) { return MicrosecondsOperator::Operation(input) / Interval::MICROS_PER_SEC; @@ -983,6 +1121,11 @@ int64_t DatePart::SecondsOperator::Operation(dtime_t input) { return MicrosecondsOperator::Operation(input) / Interval::MICROS_PER_SEC; } +template <> +int64_t DatePart::SecondsOperator::Operation(dtime_tz_t input) { + return DatePart::SecondsOperator::Operation(input.time()); +} + template <> int64_t DatePart::MinutesOperator::Operation(timestamp_t input) { auto time = Timestamp::GetTime(input); @@ -1002,6 +1145,11 @@ int64_t DatePart::MinutesOperator::Operation(dtime_t input) { return (input.micros % Interval::MICROS_PER_HOUR) / Interval::MICROS_PER_MINUTE; } +template <> +int64_t DatePart::MinutesOperator::Operation(dtime_tz_t input) { + return DatePart::MinutesOperator::Operation(input.time()); +} + template <> int64_t DatePart::HoursOperator::Operation(timestamp_t input) { return Timestamp::GetTime(input).micros / Interval::MICROS_PER_HOUR; @@ -1017,6 +1165,11 @@ int64_t DatePart::HoursOperator::Operation(dtime_t input) { return input.micros / Interval::MICROS_PER_HOUR; } +template <> +int64_t DatePart::HoursOperator::Operation(dtime_tz_t input) { + return DatePart::HoursOperator::Operation(input.time()); +} + template <> double DatePart::EpochOperator::Operation(timestamp_t input) { return Timestamp::GetEpochMicroSeconds(input) / double(Interval::MICROS_PER_SEC); @@ -1048,6 +1201,11 @@ double DatePart::EpochOperator::Operation(dtime_t input) { return input.micros / double(Interval::MICROS_PER_SEC); } +template <> +double DatePart::EpochOperator::Operation(dtime_tz_t input) { + return DatePart::EpochOperator::Operation(input.time()); +} + template <> unique_ptr DatePart::EpochOperator::PropagateStatistics(ClientContext &context, FunctionStatisticsInput &input) { @@ -1073,6 +1231,11 @@ int64_t DatePart::EraOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"era\" not recognized"); } +template <> +int64_t DatePart::EraOperator::Operation(dtime_tz_t input) { + return EraOperator::Operation(input.time()); +} + template <> int64_t DatePart::TimezoneOperator::Operation(date_t input) { throw NotImplementedException("\"date\" units \"timezone\" not recognized"); @@ -1084,8 +1247,38 @@ int64_t DatePart::TimezoneOperator::Operation(interval_t input) { } template <> -int64_t DatePart::TimezoneOperator::Operation(dtime_t input) { - return 0; +int64_t DatePart::TimezoneOperator::Operation(dtime_tz_t input) { + return input.offset(); +} + +template <> +int64_t DatePart::TimezoneHourOperator::Operation(date_t input) { + throw NotImplementedException("\"date\" units \"timezone_hour\" not recognized"); +} + +template <> +int64_t DatePart::TimezoneHourOperator::Operation(interval_t input) { + throw NotImplementedException("\"interval\" units \"timezone_hour\" not recognized"); +} + +template <> +int64_t DatePart::TimezoneHourOperator::Operation(dtime_tz_t input) { + return input.offset() / Interval::SECS_PER_HOUR; +} + +template <> +int64_t DatePart::TimezoneMinuteOperator::Operation(date_t input) { + throw NotImplementedException("\"date\" units \"timezone_minute\" not recognized"); +} + +template <> +int64_t DatePart::TimezoneMinuteOperator::Operation(interval_t input) { + throw NotImplementedException("\"interval\" units \"timezone_minute\" not recognized"); +} + +template <> +int64_t DatePart::TimezoneMinuteOperator::Operation(dtime_tz_t input) { + return (input.offset() / Interval::SECS_PER_MINUTE) % Interval::MINS_PER_HOUR; } template <> @@ -1103,6 +1296,11 @@ double DatePart::JulianDayOperator::Operation(dtime_t input) { throw NotImplementedException("\"time\" units \"julian\" not recognized"); } +template <> +double DatePart::JulianDayOperator::Operation(dtime_tz_t input) { + return JulianDayOperator::Operation(input.time()); +} + template <> void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec &double_values, const dtime_t &input, const idx_t idx, const part_mask_t mask) { @@ -1135,7 +1333,6 @@ void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec & auto part_data = HasPartValue(double_values, DatePartSpecifier::EPOCH); if (part_data) { part_data[idx] = EpochOperator::Operation(input); - ; } } @@ -1155,6 +1352,58 @@ void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec & } } +template <> +void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec &double_values, const dtime_tz_t &input, + const idx_t idx, const part_mask_t mask) { + int64_t *part_data; + if (mask & TIME) { + const auto micros = MicrosecondsOperator::Operation(input); + part_data = HasPartValue(bigint_values, DatePartSpecifier::MICROSECONDS); + if (part_data) { + part_data[idx] = micros; + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::MILLISECONDS); + if (part_data) { + part_data[idx] = micros / Interval::MICROS_PER_MSEC; + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::SECOND); + if (part_data) { + part_data[idx] = micros / Interval::MICROS_PER_SEC; + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::MINUTE); + if (part_data) { + part_data[idx] = MinutesOperator::Operation(input); + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::HOUR); + if (part_data) { + part_data[idx] = HoursOperator::Operation(input); + } + } + + if (mask & EPOCH) { + auto part_data = HasPartValue(double_values, DatePartSpecifier::EPOCH); + if (part_data) { + part_data[idx] = EpochOperator::Operation(input); + } + } + + if (mask & ZONE) { + part_data = HasPartValue(bigint_values, DatePartSpecifier::TIMEZONE); + if (part_data) { + part_data[idx] = TimezoneOperator::Operation(input); + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::TIMEZONE_HOUR); + if (part_data) { + part_data[idx] = TimezoneHourOperator::Operation(input); + } + part_data = HasPartValue(bigint_values, DatePartSpecifier::TIMEZONE_MINUTE); + if (part_data) { + part_data[idx] = TimezoneMinuteOperator::Operation(input); + } + return; + } +} + template <> void DatePart::StructOperator::Operation(bigint_vec &bigint_values, double_vec &double_values, const timestamp_t &input, const idx_t idx, const part_mask_t mask) { @@ -1292,9 +1541,11 @@ static int64_t ExtractElement(DatePartSpecifier type, T element) { case DatePartSpecifier::ERA: return DatePart::EraOperator::template Operation(element); case DatePartSpecifier::TIMEZONE: + return DatePart::TimezoneOperator::template Operation(element); case DatePartSpecifier::TIMEZONE_HOUR: + return DatePart::TimezoneHourOperator::template Operation(element); case DatePartSpecifier::TIMEZONE_MINUTE: - return DatePart::TimezoneOperator::template Operation(element); + return DatePart::TimezoneMinuteOperator::template Operation(element); default: throw NotImplementedException("Specifier type not implemented for DATEPART"); } @@ -1374,6 +1625,10 @@ static unique_ptr DatePartBind(ClientContext &context, ScalarFunct bound_function.function = DatePart::UnaryFunction; bound_function.statistics = DatePart::EpochOperator::template PropagateStatistics; break; + case LogicalType::TIME_TZ: + bound_function.function = DatePart::UnaryFunction; + bound_function.statistics = DatePart::EpochOperator::template PropagateStatistics; + break; default: throw BinderException("%s can only take temporal arguments", bound_function.name); } @@ -1407,8 +1662,9 @@ static ScalarFunctionSet GetDatePartFunction() { ScalarFunctionSet GetGenericTimePartFunction(const LogicalType &result_type, scalar_function_t date_func, scalar_function_t ts_func, scalar_function_t interval_func, - scalar_function_t time_func, function_statistics_t date_stats, - function_statistics_t ts_stats, function_statistics_t time_stats) { + scalar_function_t time_func, scalar_function_t timetz_func, + function_statistics_t date_stats, function_statistics_t ts_stats, + function_statistics_t time_stats, function_statistics_t timetz_stats) { ScalarFunctionSet operator_set; operator_set.AddFunction( ScalarFunction({LogicalType::DATE}, result_type, std::move(date_func), nullptr, nullptr, date_stats)); @@ -1417,6 +1673,8 @@ ScalarFunctionSet GetGenericTimePartFunction(const LogicalType &result_type, sca operator_set.AddFunction(ScalarFunction({LogicalType::INTERVAL}, result_type, std::move(interval_func))); operator_set.AddFunction( ScalarFunction({LogicalType::TIME}, result_type, std::move(time_func), nullptr, nullptr, time_stats)); + operator_set.AddFunction( + ScalarFunction({LogicalType::TIME_TZ}, result_type, std::move(timetz_func), nullptr, nullptr, timetz_stats)); return operator_set; } @@ -1425,8 +1683,9 @@ static ScalarFunctionSet GetTimePartFunction(const LogicalType &result_type = Lo return GetGenericTimePartFunction( result_type, DatePart::UnaryFunction, DatePart::UnaryFunction, ScalarFunction::UnaryFunction, ScalarFunction::UnaryFunction, - OP::template PropagateStatistics, OP::template PropagateStatistics, - OP::template PropagateStatistics); + ScalarFunction::UnaryFunction, OP::template PropagateStatistics, + OP::template PropagateStatistics, OP::template PropagateStatistics, + OP::template PropagateStatistics); } struct LastDayOperator { @@ -1724,7 +1983,14 @@ ScalarFunctionSet EraFun::GetFunctions() { } ScalarFunctionSet TimezoneFun::GetFunctions() { - return GetDatePartFunction(); + auto operator_set = GetDatePartFunction(); + + // PG also defines timezone(INTERVAL, TIME_TZ) => TIME_TZ + operator_set.AddFunction( + ScalarFunction({LogicalType::INTERVAL, LogicalType::TIME_TZ}, LogicalType::TIME_TZ, + DatePart::TimezoneOperator::BinaryFunction)); + + return operator_set; } ScalarFunctionSet TimezoneHourFun::GetFunctions() { @@ -1869,12 +2135,15 @@ ScalarFunctionSet DatePartFun::GetFunctions() { DatePartFunction, DatePartBind)); date_part.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::INTERVAL}, LogicalType::BIGINT, DatePartFunction, DatePartBind)); + date_part.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::TIME_TZ}, LogicalType::BIGINT, + DatePartFunction, DatePartBind)); // struct variants date_part.AddFunction(StructDatePart::GetFunction(LogicalType::DATE)); date_part.AddFunction(StructDatePart::GetFunction(LogicalType::TIMESTAMP)); date_part.AddFunction(StructDatePart::GetFunction(LogicalType::TIME)); date_part.AddFunction(StructDatePart::GetFunction(LogicalType::INTERVAL)); + date_part.AddFunction(StructDatePart::GetFunction(LogicalType::TIME_TZ)); return date_part; } diff --git a/src/duckdb/src/core_functions/scalar/date/make_date.cpp b/src/duckdb/src/core_functions/scalar/date/make_date.cpp index 2eb5248bb..b3a6a8749 100644 --- a/src/duckdb/src/core_functions/scalar/date/make_date.cpp +++ b/src/duckdb/src/core_functions/scalar/date/make_date.cpp @@ -4,6 +4,7 @@ #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/vector_operations/ternary_executor.hpp" #include "duckdb/common/vector_operations/senary_executor.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" #include diff --git a/src/duckdb/src/core_functions/scalar/date/strftime.cpp b/src/duckdb/src/core_functions/scalar/date/strftime.cpp index 708ff2c3d..72da80bb5 100644 --- a/src/duckdb/src/core_functions/scalar/date/strftime.cpp +++ b/src/duckdb/src/core_functions/scalar/date/strftime.cpp @@ -39,7 +39,7 @@ static unique_ptr StrfTimeBindFunction(ClientContext &context, Sca throw ParameterNotResolvedException(); } if (!format_arg->IsFoldable()) { - throw InvalidInputException("strftime format must be a constant"); + throw InvalidInputException(*format_arg, "strftime format must be a constant"); } Value options_str = ExpressionExecutor::EvaluateScalar(context, *format_arg); auto format_string = options_str.GetValue(); @@ -48,7 +48,7 @@ static unique_ptr StrfTimeBindFunction(ClientContext &context, Sca if (!is_null) { string error = StrTimeFormat::ParseFormatSpecifier(format_string, format); if (!error.empty()) { - throw InvalidInputException("Failed to parse format specifier %s: %s", format_string, error); + throw InvalidInputException(*format_arg, "Failed to parse format specifier %s: %s", format_string, error); } } return make_uniq(format, format_string, is_null); @@ -132,7 +132,7 @@ static unique_ptr StrpTimeBindFunction(ClientContext &context, Sca throw ParameterNotResolvedException(); } if (!arguments[1]->IsFoldable()) { - throw InvalidInputException("strptime format must be a constant"); + throw InvalidInputException(*arguments[0], "strptime format must be a constant"); } Value format_value = ExpressionExecutor::EvaluateScalar(context, *arguments[1]); string format_string; @@ -144,7 +144,7 @@ static unique_ptr StrpTimeBindFunction(ClientContext &context, Sca format.format_specifier = format_string; string error = StrTimeFormat::ParseFormatSpecifier(format_string, format); if (!error.empty()) { - throw InvalidInputException("Failed to parse format specifier %s: %s", format_string, error); + throw InvalidInputException(*arguments[0], "Failed to parse format specifier %s: %s", format_string, error); } if (format.HasFormatSpecifier(StrTimeSpecifier::UTC_OFFSET)) { bound_function.return_type = LogicalType::TIMESTAMP_TZ; @@ -153,7 +153,7 @@ static unique_ptr StrpTimeBindFunction(ClientContext &context, Sca } else if (format_value.type() == LogicalType::LIST(LogicalType::VARCHAR)) { const auto &children = ListValue::GetChildren(format_value); if (children.empty()) { - throw InvalidInputException("strptime format list must not be empty"); + throw InvalidInputException(*arguments[0], "strptime format list must not be empty"); } vector format_strings; vector formats; @@ -162,7 +162,8 @@ static unique_ptr StrpTimeBindFunction(ClientContext &context, Sca format.format_specifier = format_string; string error = StrTimeFormat::ParseFormatSpecifier(format_string, format); if (!error.empty()) { - throw InvalidInputException("Failed to parse format specifier %s: %s", format_string, error); + throw InvalidInputException(*arguments[0], "Failed to parse format specifier %s: %s", format_string, + error); } // If any format has UTC offsets, then we have to produce TSTZ if (format.HasFormatSpecifier(StrTimeSpecifier::UTC_OFFSET)) { @@ -173,7 +174,7 @@ static unique_ptr StrpTimeBindFunction(ClientContext &context, Sca } return make_uniq(formats, format_strings); } else { - throw InvalidInputException("strptime format must be a string"); + throw InvalidInputException(*arguments[0], "strptime format must be a string"); } } 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 77a0d64f7..dcbf065e4 100644 --- a/src/duckdb/src/core_functions/scalar/date/to_interval.cpp +++ b/src/duckdb/src/core_functions/scalar/date/to_interval.cpp @@ -184,13 +184,13 @@ ScalarFunction ToDaysFun::GetFunction() { } ScalarFunction ToHoursFun::GetFunction() { - return ScalarFunction({LogicalType::DOUBLE}, LogicalType::INTERVAL, - ScalarFunction::UnaryFunction); + return ScalarFunction({LogicalType::BIGINT}, LogicalType::INTERVAL, + ScalarFunction::UnaryFunction); } ScalarFunction ToMinutesFun::GetFunction() { - return ScalarFunction({LogicalType::DOUBLE}, LogicalType::INTERVAL, - ScalarFunction::UnaryFunction); + return ScalarFunction({LogicalType::BIGINT}, LogicalType::INTERVAL, + ScalarFunction::UnaryFunction); } ScalarFunction ToSecondsFun::GetFunction() { diff --git a/src/duckdb/src/core_functions/scalar/generic/error.cpp b/src/duckdb/src/core_functions/scalar/generic/error.cpp index 5d38236bf..9c172e877 100644 --- a/src/duckdb/src/core_functions/scalar/generic/error.cpp +++ b/src/duckdb/src/core_functions/scalar/generic/error.cpp @@ -6,15 +6,15 @@ namespace duckdb { struct ErrorOperator { template static inline TR Operation(const TA &input) { - throw Exception(input.GetString()); + throw InvalidInputException(input.GetString()); } }; ScalarFunction ErrorFun::GetFunction() { - auto fun = ScalarFunction({LogicalType::VARCHAR}, LogicalType::BOOLEAN, - ScalarFunction::UnaryFunction); + auto fun = ScalarFunction({LogicalType::VARCHAR}, LogicalType::SQLNULL, + ScalarFunction::UnaryFunction); // Set the function with side effects to avoid the optimization. - fun.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + fun.stability = FunctionStability::VOLATILE; return fun; } diff --git a/src/duckdb/src/core_functions/scalar/generic/least.cpp b/src/duckdb/src/core_functions/scalar/generic/least.cpp index 42e2d68d0..16f859bf5 100644 --- a/src/duckdb/src/core_functions/scalar/generic/least.cpp +++ b/src/duckdb/src/core_functions/scalar/generic/least.cpp @@ -97,7 +97,7 @@ static void LeastGreatestFunction(DataChunk &args, ExpressionState &state, Vecto template ScalarFunction GetLeastGreatestFunction(const LogicalType &type) { return ScalarFunction({type}, type, LeastGreatestFunction, nullptr, nullptr, nullptr, nullptr, type, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING); } template @@ -105,17 +105,16 @@ static ScalarFunctionSet GetLeastGreatestFunctions() { ScalarFunctionSet fun_set; fun_set.AddFunction(ScalarFunction({LogicalType::BIGINT}, LogicalType::BIGINT, LeastGreatestFunction, nullptr, nullptr, nullptr, nullptr, LogicalType::BIGINT, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); fun_set.AddFunction(ScalarFunction( {LogicalType::HUGEINT}, LogicalType::HUGEINT, LeastGreatestFunction, nullptr, nullptr, nullptr, - nullptr, LogicalType::HUGEINT, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + nullptr, LogicalType::HUGEINT, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); fun_set.AddFunction(ScalarFunction({LogicalType::DOUBLE}, LogicalType::DOUBLE, LeastGreatestFunction, nullptr, nullptr, nullptr, nullptr, LogicalType::DOUBLE, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); - fun_set.AddFunction(ScalarFunction({LogicalType::VARCHAR}, LogicalType::VARCHAR, - LeastGreatestFunction, nullptr, nullptr, nullptr, nullptr, - LogicalType::VARCHAR, FunctionSideEffects::NO_SIDE_EFFECTS, - FunctionNullHandling::SPECIAL_HANDLING)); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); + fun_set.AddFunction(ScalarFunction( + {LogicalType::VARCHAR}, LogicalType::VARCHAR, LeastGreatestFunction, nullptr, nullptr, + nullptr, nullptr, LogicalType::VARCHAR, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); fun_set.AddFunction(GetLeastGreatestFunction(LogicalType::TIMESTAMP)); fun_set.AddFunction(GetLeastGreatestFunction(LogicalType::TIME)); diff --git a/src/duckdb/src/core_functions/scalar/generic/stats.cpp b/src/duckdb/src/core_functions/scalar/generic/stats.cpp index d19dcfc9f..f547ca78d 100644 --- a/src/duckdb/src/core_functions/scalar/generic/stats.cpp +++ b/src/duckdb/src/core_functions/scalar/generic/stats.cpp @@ -47,7 +47,7 @@ ScalarFunction StatsFun::GetFunction() { ScalarFunction stats({LogicalType::ANY}, LogicalType::VARCHAR, StatsFunction, StatsBind, nullptr, StatsPropagateStats); stats.null_handling = FunctionNullHandling::SPECIAL_HANDLING; - stats.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + stats.stability = FunctionStability::VOLATILE; return stats; } diff --git a/src/duckdb/src/core_functions/scalar/generic/system_functions.cpp b/src/duckdb/src/core_functions/scalar/generic/system_functions.cpp index 1bfb8555b..2abb40599 100644 --- a/src/duckdb/src/core_functions/scalar/generic/system_functions.cpp +++ b/src/duckdb/src/core_functions/scalar/generic/system_functions.cpp @@ -76,29 +76,40 @@ static void VersionFunction(DataChunk &input, ExpressionState &state, Vector &re ScalarFunction CurrentQueryFun::GetFunction() { ScalarFunction current_query({}, LogicalType::VARCHAR, CurrentQueryFunction); - current_query.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + current_query.stability = FunctionStability::VOLATILE; return current_query; } ScalarFunction CurrentSchemaFun::GetFunction() { - return ScalarFunction({}, LogicalType::VARCHAR, CurrentSchemaFunction); + ScalarFunction current_schema({}, LogicalType::VARCHAR, CurrentSchemaFunction); + current_schema.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; + return current_schema; } ScalarFunction CurrentDatabaseFun::GetFunction() { - return ScalarFunction({}, LogicalType::VARCHAR, CurrentDatabaseFunction); + ScalarFunction current_database({}, LogicalType::VARCHAR, CurrentDatabaseFunction); + current_database.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; + return current_database; } ScalarFunction CurrentSchemasFun::GetFunction() { auto varchar_list_type = LogicalType::LIST(LogicalType::VARCHAR); - return ScalarFunction({LogicalType::BOOLEAN}, varchar_list_type, CurrentSchemasFunction); + ScalarFunction current_schemas({LogicalType::BOOLEAN}, varchar_list_type, CurrentSchemasFunction); + current_schemas.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; + return current_schemas; } ScalarFunction InSearchPathFun::GetFunction() { - return ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, InSearchPathFunction); + ScalarFunction in_search_path({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, + InSearchPathFunction); + in_search_path.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; + return in_search_path; } ScalarFunction CurrentTransactionIdFun::GetFunction() { - return ScalarFunction({}, LogicalType::BIGINT, TransactionIdCurrent); + ScalarFunction txid_current({}, LogicalType::BIGINT, TransactionIdCurrent); + txid_current.stability = FunctionStability::CONSISTENT_WITHIN_QUERY; + return txid_current; } ScalarFunction VersionFun::GetFunction() { 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 4539b4e1f..144e2f504 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_aggregates.cpp @@ -452,9 +452,8 @@ static unique_ptr ListAggregatesBind(ClientContext &context, Scala } // look up the aggregate function in the catalog - QueryErrorContext error_context(nullptr, 0); - auto &func = Catalog::GetSystemCatalog(context).GetEntry( - context, DEFAULT_SCHEMA, function_name, error_context); + auto &func = Catalog::GetSystemCatalog(context).GetEntry(context, DEFAULT_SCHEMA, + function_name); D_ASSERT(func.type == CatalogType::AGGREGATE_FUNCTION_ENTRY); if (is_parameter) { @@ -464,7 +463,7 @@ static unique_ptr ListAggregatesBind(ClientContext &context, Scala } // find a matching aggregate function - string error; + ErrorData error; vector types; types.push_back(child_type); // push any extra arguments into the type list @@ -475,7 +474,7 @@ static unique_ptr ListAggregatesBind(ClientContext &context, Scala FunctionBinder function_binder(context); auto best_function_idx = function_binder.BindFunction(func.name, func.functions, types, error); if (best_function_idx == DConstants::INVALID_INDEX) { - throw BinderException("No matching aggregate function\n%s", error); + throw BinderException("No matching aggregate function\n%s", error.Message()); } // found a matching function, bind it as an aggregate diff --git a/src/duckdb/src/core_functions/scalar/list/list_reduce.cpp b/src/duckdb/src/core_functions/scalar/list/list_reduce.cpp new file mode 100644 index 000000000..1e276e712 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/list/list_reduce.cpp @@ -0,0 +1,230 @@ +#include "duckdb/core_functions/scalar/list_functions.hpp" +#include "duckdb/core_functions/lambda_functions.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" + +namespace duckdb { + +struct ReduceExecuteInfo { + ReduceExecuteInfo(LambdaFunctions::LambdaInfo &info, ClientContext &context) : left_slice(*info.child_vector) { + SelectionVector left_vector(info.row_count); + active_rows.Resize(0, info.row_count); + active_rows.SetAllValid(info.row_count); + + right_sel.Initialize(info.row_count); + left_sel.Initialize(info.row_count); + active_rows_sel.Initialize(info.row_count); + + idx_t reduced_row_idx = 0; + + for (idx_t original_row_idx = 0; original_row_idx < info.row_count; original_row_idx++) { + auto list_column_format_index = info.list_column_format.sel->get_index(original_row_idx); + if (info.list_column_format.validity.RowIsValid(list_column_format_index)) { + if (info.list_entries[list_column_format_index].length == 0) { + throw ParameterNotAllowedException("Cannot perform list_reduce on an empty input list"); + } + left_vector.set_index(reduced_row_idx, info.list_entries[list_column_format_index].offset); + reduced_row_idx++; + } else { + // Remove the invalid rows + info.result_validity->SetInvalid(original_row_idx); + active_rows.SetInvalid(original_row_idx); + } + } + + left_slice.Slice(left_vector, reduced_row_idx); + + if (info.has_index) { + input_types.push_back(LogicalType::BIGINT); + } + input_types.push_back(left_slice.GetType()); + input_types.push_back(left_slice.GetType()); + for (auto &entry : info.column_infos) { + input_types.push_back(entry.vector.get().GetType()); + } + + expr_executor = make_uniq(context, *info.lambda_expr); + }; + ValidityMask active_rows; + Vector left_slice; + unique_ptr expr_executor; + vector input_types; + + SelectionVector right_sel; + SelectionVector left_sel; + SelectionVector active_rows_sel; +}; + +static bool ExecuteReduce(idx_t loops, ReduceExecuteInfo &execute_info, LambdaFunctions::LambdaInfo &info, + DataChunk &result_chunk) { + idx_t original_row_idx = 0; + idx_t reduced_row_idx = 0; + idx_t valid_row_idx = 0; + + // create selection vectors for the left and right slice + auto data = execute_info.active_rows.GetData(); + + idx_t bits_per_entry = sizeof(idx_t) * 8; + for (idx_t entry_idx = 0; original_row_idx < info.row_count; entry_idx++) { + if (data[entry_idx] == 0) { + original_row_idx += bits_per_entry; + continue; + } + + for (idx_t j = 0; entry_idx * bits_per_entry + j < info.row_count; j++) { + if (!execute_info.active_rows.RowIsValid(original_row_idx)) { + original_row_idx++; + continue; + } + auto list_column_format_index = info.list_column_format.sel->get_index(original_row_idx); + if (info.list_entries[list_column_format_index].length > loops + 1) { + execute_info.right_sel.set_index(reduced_row_idx, + info.list_entries[list_column_format_index].offset + loops + 1); + execute_info.left_sel.set_index(reduced_row_idx, valid_row_idx); + execute_info.active_rows_sel.set_index(reduced_row_idx, original_row_idx); + + reduced_row_idx++; + } else { + execute_info.active_rows.SetInvalid(original_row_idx); + info.result.SetValue(original_row_idx, execute_info.left_slice.GetValue(valid_row_idx)); + } + original_row_idx++; + valid_row_idx++; + } + } + + if (reduced_row_idx == 0) { + return true; + } + + // create the index vector + Vector index_vector(Value::BIGINT(loops + 1)); + + // slice the left and right slice + execute_info.left_slice.Slice(execute_info.left_slice, execute_info.left_sel, reduced_row_idx); + Vector right_slice(*info.child_vector, execute_info.right_sel, reduced_row_idx); + + // create the input chunk + DataChunk input_chunk; + input_chunk.InitializeEmpty(execute_info.input_types); + input_chunk.SetCardinality(reduced_row_idx); + + idx_t slice_offset = info.has_index ? 1 : 0; + if (info.has_index) { + input_chunk.data[0].Reference(index_vector); + } + input_chunk.data[slice_offset + 1].Reference(execute_info.left_slice); + input_chunk.data[slice_offset].Reference(right_slice); + + // add the other columns + vector slices; + for (idx_t i = 0; i < info.column_infos.size(); i++) { + if (info.column_infos[i].vector.get().GetVectorType() == VectorType::CONSTANT_VECTOR) { + // only reference constant vectors + input_chunk.data[slice_offset + 2 + i].Reference(info.column_infos[i].vector); + } else { + // slice the other vectors + slices.emplace_back(info.column_infos[i].vector, execute_info.active_rows_sel, reduced_row_idx); + input_chunk.data[slice_offset + 2 + i].Reference(slices.back()); + } + } + + result_chunk.Reset(); + result_chunk.SetCardinality(reduced_row_idx); + + execute_info.expr_executor->Execute(input_chunk, result_chunk); + + // use the result chunk to update the left slice + execute_info.left_slice.Reference(result_chunk.data[0]); + return false; +} + +void LambdaFunctions::ListReduceFunction(duckdb::DataChunk &args, duckdb::ExpressionState &state, + duckdb::Vector &result) { + // Initializes the left slice from the list entries, active rows, the expression executor and the input types + bool completed = false; + LambdaFunctions::LambdaInfo info(args, state, result, completed); + if (completed) { + return; + } + + ReduceExecuteInfo execute_info(info, state.GetContext()); + + // Since the left slice references the result chunk, we need to create two result chunks. + // This means there is always an empty result chunk for the next iteration, + // without the referenced chunk having to be reset until the current iteration is complete. + DataChunk odd_result_chunk; + odd_result_chunk.Initialize(Allocator::DefaultAllocator(), {info.lambda_expr->return_type}); + + DataChunk even_result_chunk; + even_result_chunk.Initialize(Allocator::DefaultAllocator(), {info.lambda_expr->return_type}); + + idx_t loops = 0; + bool end = false; + // Execute reduce until all rows are finished + while (!end) { + auto &result_chunk = loops % 2 ? odd_result_chunk : even_result_chunk; + auto &spare_result_chunk = loops % 2 ? even_result_chunk : odd_result_chunk; + + end = ExecuteReduce(loops, execute_info, info, result_chunk); + spare_result_chunk.Reset(); + + loops++; + } + + if (info.is_all_constant && !info.is_volatile) { + info.result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +static unique_ptr ListReduceBind(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(); + if (bound_lambda_expr.parameter_count < 2 || bound_lambda_expr.parameter_count > 3) { + throw BinderException("list_reduce expects a function with 2 or 3 arguments"); + } + auto has_index = bound_lambda_expr.parameter_count == 3; + + unique_ptr bind_data = LambdaFunctions::ListLambdaPrepareBind(arguments, context, bound_function); + if (bind_data) { + return bind_data; + } + + auto list_child_type = arguments[0]->return_type; + list_child_type = ListType::GetChildType(list_child_type); + + auto cast_lambda_expr = + BoundCastExpression::AddCastToType(context, std::move(bound_lambda_expr.lambda_expr), list_child_type, false); + if (!cast_lambda_expr) { + throw BinderException("Could not cast lambda expression to list child type"); + } + bound_function.return_type = cast_lambda_expr->return_type; + return make_uniq(bound_function.return_type, std::move(cast_lambda_expr), has_index); +} + +static LogicalType ListReduceBindLambda(const idx_t parameter_idx, const LogicalType &list_child_type) { + return LambdaFunctions::BindTernaryLambda(parameter_idx, list_child_type); +} + +ScalarFunction ListReduceFun::GetFunction() { + ScalarFunction fun({LogicalType::LIST(LogicalType::ANY), LogicalType::LAMBDA}, LogicalType::ANY, + LambdaFunctions::ListReduceFunction, ListReduceBind, nullptr, nullptr); + + fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + fun.serialize = ListLambdaBindData::Serialize; + fun.deserialize = ListLambdaBindData::Deserialize; + fun.bind_lambda = ListReduceBindLambda; + + 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 340ffc5d8..85d3ceb55 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_sort.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_sort.cpp @@ -1,6 +1,6 @@ #include "duckdb/core_functions/scalar/list_functions.hpp" #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" diff --git a/src/duckdb/src/core_functions/scalar/list/list_value.cpp b/src/duckdb/src/core_functions/scalar/list/list_value.cpp index ea60d8212..9d9c21c65 100644 --- a/src/duckdb/src/core_functions/scalar/list/list_value.cpp +++ b/src/duckdb/src/core_functions/scalar/list/list_value.cpp @@ -7,6 +7,7 @@ #include "duckdb/storage/statistics/list_stats.hpp" #include "duckdb/planner/expression_binder.hpp" #include "duckdb/function/scalar/nested_functions.hpp" +#include "duckdb/parser/query_error_context.hpp" namespace duckdb { @@ -36,10 +37,28 @@ static void ListValueFunction(DataChunk &args, ExpressionState &state, Vector &r static unique_ptr ListValueBind(ClientContext &context, ScalarFunction &bound_function, vector> &arguments) { // collect names and deconflict, construct return type - LogicalType child_type = arguments.empty() ? LogicalType::SQLNULL : arguments[0]->return_type; + LogicalType child_type = + arguments.empty() ? LogicalType::SQLNULL : ExpressionBinder::GetExpressionReturnType(*arguments[0]); for (idx_t i = 1; i < arguments.size(); i++) { - child_type = LogicalType::MaxLogicalType(child_type, arguments[i]->return_type); + auto arg_type = ExpressionBinder::GetExpressionReturnType(*arguments[i]); + if (!LogicalType::TryGetMaxLogicalType(context, child_type, arg_type, child_type)) { + string list_arguments = "Full list: "; + idx_t error_index = list_arguments.size(); + for (idx_t k = 0; k < arguments.size(); k++) { + if (k > 0) { + list_arguments += ", "; + } + if (k == i) { + error_index = list_arguments.size(); + } + list_arguments += arguments[k]->ToString() + " " + arguments[k]->return_type.ToString(); + } + auto error = StringUtil::Format("Cannot create a list of types %s and %s - an explicit cast is required", + child_type.ToString(), arg_type.ToString()); + throw BinderException(QueryErrorContext::Format(list_arguments, error, int(error_index), false)); + } } + child_type = LogicalType::NormalizeType(child_type); // this is more for completeness reasons bound_function.varargs = child_type; diff --git a/src/duckdb/src/core_functions/scalar/map/map.cpp b/src/duckdb/src/core_functions/scalar/map/map.cpp index c56d15f9d..c92483d4d 100644 --- a/src/duckdb/src/core_functions/scalar/map/map.cpp +++ b/src/duckdb/src/core_functions/scalar/map/map.cpp @@ -32,12 +32,13 @@ static void AlignVectorToReference(const Vector &original, const Vector &referen Vector expanded_const(ListType::GetChildType(original.GetType()), new_length); - auto expansion_factor = new_length / original_length; - if (expansion_factor != tuple_count) { + if (new_length != tuple_count * original_length) { throw InvalidInputException("Error in MAP creation: key list and value list do not align. i.e. different " "size or incompatible structure"); } + auto expansion_factor = original_length ? new_length / original_length : original_length; CreateExpandedVector(original, expanded_const, expansion_factor); + result.Reference(expanded_const); } @@ -186,14 +187,14 @@ static unique_ptr MapBind(ClientContext &context, ScalarFunction & child_list_t child_types; if (arguments.size() != 2 && !arguments.empty()) { - throw Exception("We need exactly two lists for a map"); + throw InvalidInputException("We need exactly two lists for a map"); } if (arguments.size() == 2) { if (arguments[0]->return_type.id() != LogicalTypeId::LIST) { - throw Exception("First argument is not a list"); + throw InvalidInputException("First argument is not a list"); } if (arguments[1]->return_type.id() != LogicalTypeId::LIST) { - throw Exception("Second argument is not a list"); + throw InvalidInputException("Second argument is not a list"); } child_types.push_back(make_pair("key", arguments[0]->return_type)); child_types.push_back(make_pair("value", arguments[1]->return_type)); diff --git a/src/duckdb/src/core_functions/scalar/math/numeric.cpp b/src/duckdb/src/core_functions/scalar/math/numeric.cpp index 7fdc02aca..22894b286 100644 --- a/src/duckdb/src/core_functions/scalar/math/numeric.cpp +++ b/src/duckdb/src/core_functions/scalar/math/numeric.cpp @@ -346,7 +346,7 @@ struct CeilDecimalOperator { static void Operation(DataChunk &input, uint8_t scale, Vector &result) { T power_of_ten = POWERS_OF_TEN_CLASS::POWERS_OF_TEN[scale]; UnaryExecutor::Execute(input.data[0], result, input.size(), [&](T input) { - if (input < 0) { + if (input <= 0) { // below 0 we floor the number (e.g. -10.5 -> -10) return input / power_of_ten; } else { @@ -484,6 +484,7 @@ ScalarFunctionSet TruncFun::GetFunctions() { case LogicalTypeId::USMALLINT: case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: + case LogicalTypeId::UHUGEINT: func = ScalarFunction::NopFunction; break; default: @@ -1067,7 +1068,7 @@ struct ASinOperator { template static inline TR Operation(TA input) { if (input < -1 || input > 1) { - throw Exception("ASIN is undefined outside [-1,1]"); + throw InvalidInputException("ASIN is undefined outside [-1,1]"); } return (double)std::asin(input); } @@ -1199,7 +1200,9 @@ struct FactorialOperator { static inline TR Operation(TA left) { TR ret = 1; for (TA i = 2; i <= left; i++) { - ret *= i; + if (!TryMultiplyOperator::Operation(ret, TR(i), ret)) { + throw OutOfRangeException("Value out of range"); + } } return ret; } diff --git a/src/duckdb/src/core_functions/scalar/operators/bitwise.cpp b/src/duckdb/src/core_functions/scalar/operators/bitwise.cpp index f1604aa69..65781f8d0 100644 --- a/src/duckdb/src/core_functions/scalar/operators/bitwise.cpp +++ b/src/duckdb/src/core_functions/scalar/operators/bitwise.cpp @@ -36,6 +36,9 @@ static scalar_function_t GetScalarIntegerUnaryFunction(const LogicalType &type) case LogicalTypeId::HUGEINT: function = &ScalarFunction::UnaryFunction; break; + case LogicalTypeId::UHUGEINT: + function = &ScalarFunction::UnaryFunction; + break; default: throw NotImplementedException("Unimplemented type for GetScalarIntegerUnaryFunction"); } @@ -73,6 +76,9 @@ static scalar_function_t GetScalarIntegerBinaryFunction(const LogicalType &type) case LogicalTypeId::HUGEINT: function = &ScalarFunction::BinaryFunction; break; + case LogicalTypeId::UHUGEINT: + function = &ScalarFunction::BinaryFunction; + break; default: throw NotImplementedException("Unimplemented type for GetScalarIntegerBinaryFunction"); } diff --git a/src/duckdb/src/core_functions/scalar/random/random.cpp b/src/duckdb/src/core_functions/scalar/random/random.cpp index df2f35534..02567a486 100644 --- a/src/duckdb/src/core_functions/scalar/random/random.cpp +++ b/src/duckdb/src/core_functions/scalar/random/random.cpp @@ -36,7 +36,7 @@ static unique_ptr RandomInitLocalState(ExpressionState &stat ScalarFunction RandomFun::GetFunction() { ScalarFunction random("random", {}, LogicalType::DOUBLE, RandomFunction, nullptr, nullptr, nullptr, RandomInitLocalState); - random.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + random.stability = FunctionStability::VOLATILE; return random; } @@ -56,7 +56,7 @@ ScalarFunction UUIDFun::GetFunction() { ScalarFunction uuid_function({}, LogicalType::UUID, GenerateUUIDFunction, nullptr, nullptr, nullptr, RandomInitLocalState); // generate a random uuid - uuid_function.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + uuid_function.stability = FunctionStability::VOLATILE; return uuid_function; } diff --git a/src/duckdb/src/core_functions/scalar/random/setseed.cpp b/src/duckdb/src/core_functions/scalar/random/setseed.cpp index 24f460eb7..f2db16e6c 100644 --- a/src/duckdb/src/core_functions/scalar/random/setseed.cpp +++ b/src/duckdb/src/core_functions/scalar/random/setseed.cpp @@ -37,7 +37,7 @@ static void SetSeedFunction(DataChunk &args, ExpressionState &state, Vector &res auto &random_engine = RandomEngine::Get(info.context); for (idx_t i = 0; i < args.size(); i++) { if (input_seeds[i] < -1.0 || input_seeds[i] > 1.0 || Value::IsNan(input_seeds[i])) { - throw Exception("SETSEED accepts seed values between -1.0 and 1.0, inclusive"); + throw InvalidInputException("SETSEED accepts seed values between -1.0 and 1.0, inclusive"); } uint32_t norm_seed = (input_seeds[i] + 1.0) * half_max; random_engine.SetSeed(norm_seed); @@ -54,7 +54,7 @@ unique_ptr SetSeedBind(ClientContext &context, ScalarFunction &bou ScalarFunction SetseedFun::GetFunction() { ScalarFunction setseed("setseed", {LogicalType::DOUBLE}, LogicalType::SQLNULL, SetSeedFunction, SetSeedBind); - setseed.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + setseed.stability = FunctionStability::VOLATILE; return setseed; } diff --git a/src/duckdb/src/core_functions/scalar/secret/which_secret.cpp b/src/duckdb/src/core_functions/scalar/secret/which_secret.cpp new file mode 100644 index 000000000..dfa54e627 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/secret/which_secret.cpp @@ -0,0 +1,28 @@ +#include "duckdb/core_functions/scalar/secret_functions.hpp" +#include "duckdb/main/secret/secret_manager.hpp" + +namespace duckdb { + +static void WhichSecretFunction(DataChunk &args, ExpressionState &state, Vector &result) { + D_ASSERT(args.ColumnCount() == 2); + + auto &secret_manager = SecretManager::Get(state.GetContext()); + auto transaction = CatalogTransaction::GetSystemCatalogTransaction(state.GetContext()); + + BinaryExecutor::Execute( + args.data[0], args.data[1], result, args.size(), [&](string_t path, string_t type) { + auto secret_match = secret_manager.LookupSecret(transaction, path.GetString(), type.GetString()); + if (!secret_match.HasMatch()) { + return string_t(); + } + return StringVector::AddString(result, secret_match.GetSecret().GetName()); + }); +} + +ScalarFunction WhichSecretFun::GetFunction() { + ScalarFunction which_secret("which_secret", {LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::VARCHAR, + WhichSecretFunction, nullptr, nullptr, nullptr, nullptr); + return which_secret; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/string/bar.cpp b/src/duckdb/src/core_functions/scalar/string/bar.cpp index cf194fd4c..291553a3a 100644 --- a/src/duckdb/src/core_functions/scalar/string/bar.cpp +++ b/src/duckdb/src/core_functions/scalar/string/bar.cpp @@ -15,13 +15,13 @@ static string_t BarScalarFunction(double x, double min, double max, double max_w static const idx_t PARTIAL_BLOCKS_COUNT = UnicodeBar::PartialBlocksCount(); if (!Value::IsFinite(max_width)) { - throw ValueOutOfRangeException("Max bar width must not be NaN or infinity"); + throw OutOfRangeException("Max bar width must not be NaN or infinity"); } if (max_width < 1) { - throw ValueOutOfRangeException("Max bar width must be >= 1"); + throw OutOfRangeException("Max bar width must be >= 1"); } if (max_width > 1000) { - throw ValueOutOfRangeException("Max bar width must be <= 1000"); + throw OutOfRangeException("Max bar width must be <= 1000"); } double width; @@ -35,7 +35,7 @@ static string_t BarScalarFunction(double x, double min, double max, double max_w } if (!Value::IsFinite(width)) { - throw ValueOutOfRangeException("Bar width must not be NaN or infinity"); + throw OutOfRangeException("Bar width must not be NaN or infinity"); } result.clear(); diff --git a/src/duckdb/src/core_functions/scalar/string/hex.cpp b/src/duckdb/src/core_functions/scalar/string/hex.cpp index ffea5e316..eac44555b 100644 --- a/src/duckdb/src/core_functions/scalar/string/hex.cpp +++ b/src/duckdb/src/core_functions/scalar/string/hex.cpp @@ -19,7 +19,8 @@ static void WriteHexBytes(uint64_t x, char *&output, idx_t buffer_size) { } } -static void WriteHugeIntHexBytes(hugeint_t x, char *&output, idx_t buffer_size) { +template +static void WriteHugeIntHexBytes(T x, char *&output, idx_t buffer_size) { idx_t offset = buffer_size * 4; auto upper = x.upper; auto lower = x.lower; @@ -45,7 +46,8 @@ static void WriteBinBytes(uint64_t x, char *&output, idx_t buffer_size) { } } -static void WriteHugeIntBinBytes(hugeint_t x, char *&output, idx_t buffer_size) { +template +static void WriteHugeIntBinBytes(T x, char *&output, idx_t buffer_size) { auto upper = x.upper; auto lower = x.lower; idx_t offset = buffer_size; @@ -133,7 +135,34 @@ struct HexHugeIntOperator { auto target = StringVector::EmptyString(result, buffer_size); auto output = target.GetDataWriteable(); - WriteHugeIntHexBytes(input, output, buffer_size); + WriteHugeIntHexBytes(input, output, buffer_size); + + target.Finalize(); + return target; + } +}; + +struct HexUhugeIntOperator { + template + static RESULT_TYPE Operation(INPUT_TYPE input, Vector &result) { + + idx_t num_leading_zero = CountZeros::Leading(input); + idx_t buffer_size = sizeof(INPUT_TYPE) * 2 - (num_leading_zero / 4); + + // Special case: All bits are zero + if (buffer_size == 0) { + auto target = StringVector::EmptyString(result, 1); + auto output = target.GetDataWriteable(); + *output = '0'; + target.Finalize(); + return target; + } + + D_ASSERT(buffer_size > 0); + auto target = StringVector::EmptyString(result, buffer_size); + auto output = target.GetDataWriteable(); + + WriteHugeIntHexBytes(input, output, buffer_size); target.Finalize(); return target; @@ -219,7 +248,32 @@ struct BinaryHugeIntOperator { auto target = StringVector::EmptyString(result, buffer_size); auto output = target.GetDataWriteable(); - WriteHugeIntBinBytes(input, output, buffer_size); + WriteHugeIntBinBytes(input, output, buffer_size); + + target.Finalize(); + return target; + } +}; + +struct BinaryUhugeIntOperator { + template + static RESULT_TYPE Operation(INPUT_TYPE input, Vector &result) { + idx_t num_leading_zero = CountZeros::Leading(input); + idx_t buffer_size = sizeof(INPUT_TYPE) * 8 - num_leading_zero; + + // Special case: All bits are zero + if (buffer_size == 0) { + auto target = StringVector::EmptyString(result, 1); + auto output = target.GetDataWriteable(); + *output = '0'; + target.Finalize(); + return target; + } + + auto target = StringVector::EmptyString(result, buffer_size); + auto output = target.GetDataWriteable(); + + WriteHugeIntBinBytes(input, output, buffer_size); target.Finalize(); return target; @@ -346,6 +400,9 @@ ScalarFunctionSet HexFun::GetFunctions() { to_hex.AddFunction( ScalarFunction({LogicalType::HUGEINT}, LogicalType::VARCHAR, ToHexFunction)); + + to_hex.AddFunction( + ScalarFunction({LogicalType::UHUGEINT}, LogicalType::VARCHAR, ToHexFunction)); return to_hex; } @@ -364,6 +421,8 @@ ScalarFunctionSet BinFun::GetFunctions() { ScalarFunction({LogicalType::BIGINT}, LogicalType::VARCHAR, ToBinaryFunction)); to_binary.AddFunction(ScalarFunction({LogicalType::HUGEINT}, LogicalType::VARCHAR, ToBinaryFunction)); + to_binary.AddFunction(ScalarFunction({LogicalType::UHUGEINT}, LogicalType::VARCHAR, + ToBinaryFunction)); return to_binary; } diff --git a/src/duckdb/src/core_functions/scalar/string/pad.cpp b/src/duckdb/src/core_functions/scalar/string/pad.cpp index 3ff111ca6..8ad16f64e 100644 --- a/src/duckdb/src/core_functions/scalar/string/pad.cpp +++ b/src/duckdb/src/core_functions/scalar/string/pad.cpp @@ -71,7 +71,7 @@ static string_t LeftPadFunction(const string_t &str, const int32_t len, const st // Left pad by the number of characters still needed if (!InsertPadding(len - written.second, pad, result)) { - throw Exception("Insufficient padding in LPAD."); + throw InvalidInputException("Insufficient padding in LPAD."); } // Append as much of the original string as fits @@ -103,7 +103,7 @@ static string_t RightPadFunction(const string_t &str, const int32_t len, const s // Right pad by the number of characters still needed if (!InsertPadding(len - written.second, pad, result)) { - throw Exception("Insufficient padding in RPAD."); + throw InvalidInputException("Insufficient padding in RPAD."); }; return string_t(result.data(), result.size()); diff --git a/src/duckdb/src/core_functions/scalar/string/parse_path.cpp b/src/duckdb/src/core_functions/scalar/string/parse_path.cpp new file mode 100644 index 000000000..e6ce3066a --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/string/parse_path.cpp @@ -0,0 +1,348 @@ +#include "duckdb/core_functions/scalar/string_functions.hpp" +#include "duckdb/function/scalar/string_functions.hpp" +#include "duckdb/common/local_file_system.hpp" +#include + +namespace duckdb { + +static string GetSeparator(const string_t &input) { + string option = input.GetString(); + + // system's path separator + auto fs = FileSystem::CreateLocal(); + auto system_sep = fs->PathSeparator(option); + + string separator; + if (option == "system") { + separator = system_sep; + } else if (option == "forward_slash") { + separator = "/"; + } else if (option == "backslash") { + separator = "\\"; + } else { // both_slash (default) + separator = "/\\"; + } + return separator; +} + +struct SplitInput { + SplitInput(Vector &result_list, Vector &result_child, idx_t offset) + : result_list(result_list), result_child(result_child), offset(offset) { + } + + Vector &result_list; + Vector &result_child; + idx_t offset; + + void AddSplit(const char *split_data, idx_t split_size, idx_t list_idx) { + auto list_entry = offset + list_idx; + if (list_entry >= ListVector::GetListCapacity(result_list)) { + ListVector::SetListSize(result_list, offset + list_idx); + ListVector::Reserve(result_list, ListVector::GetListCapacity(result_list) * 2); + } + FlatVector::GetData(result_child)[list_entry] = + StringVector::AddString(result_child, split_data, split_size); + } +}; + +static bool IsIdxValid(const idx_t &i, const idx_t &sentence_size) { + if (i > sentence_size || i == DConstants::INVALID_INDEX) { + return false; + } + return true; +} + +static idx_t Find(const char *input_data, idx_t input_size, const string &sep_data) { + if (sep_data.empty()) { + return 0; + } + auto pos = ContainsFun::Find(const_uchar_ptr_cast(input_data), input_size, const_uchar_ptr_cast(&sep_data[0]), 1); + // both_slash option + if (sep_data.size() > 1) { + auto sec_pos = + ContainsFun::Find(const_uchar_ptr_cast(input_data), input_size, const_uchar_ptr_cast(&sep_data[1]), 1); + // choose the leftmost valid position + if (sec_pos != DConstants::INVALID_INDEX && (sec_pos < pos || pos == DConstants::INVALID_INDEX)) { + return sec_pos; + } + } + return pos; +} + +static idx_t FindLast(const char *data_ptr, idx_t input_size, const string &sep_data) { + idx_t start = 0; + while (input_size > 0) { + auto pos = Find(data_ptr, input_size, sep_data); + if (!IsIdxValid(pos, input_size)) { + break; + } + start += (pos + 1); + data_ptr += (pos + 1); + input_size -= (pos + 1); + } + if (start < 1) { + return DConstants::INVALID_INDEX; + } + return start - 1; +} + +static idx_t SplitPath(string_t input, const string &sep, SplitInput &state) { + auto input_data = input.GetData(); + auto input_size = input.GetSize(); + if (!input_size) { + return 0; + } + idx_t list_idx = 0; + while (input_size > 0) { + auto pos = Find(input_data, input_size, sep); + if (!IsIdxValid(pos, input_size)) { + break; + } + + D_ASSERT(input_size >= pos); + if (pos == 0) { + if (list_idx == 0) { // first character in path is separator + state.AddSplit(input_data, 1, list_idx); + list_idx++; + if (input_size == 1) { // special case: the only character in path is a separator + return list_idx; + } + } // else: separator is in the path + } else { + state.AddSplit(input_data, pos, list_idx); + list_idx++; + } + input_data += (pos + 1); + input_size -= (pos + 1); + } + if (input_size > 0) { + state.AddSplit(input_data, input_size, list_idx); + list_idx++; + } + return list_idx; +} + +static void ReadOptionalArgs(DataChunk &args, Vector &sep, Vector &trim, const bool &front_trim) { + switch (args.ColumnCount()) { + case 1: { + // use default values + break; + } + case 2: { + UnifiedVectorFormat sec_arg; + args.data[1].ToUnifiedFormat(args.size(), sec_arg); + if (sec_arg.validity.RowIsValid(0)) { // if not NULL + switch (args.data[1].GetType().id()) { + case LogicalTypeId::VARCHAR: { + sep.Reinterpret(args.data[1]); + break; + } + case LogicalTypeId::BOOLEAN: { // parse_path and parse_driname won't get in here + trim.Reinterpret(args.data[1]); + break; + } + default: + throw InvalidInputException("Invalid argument type"); + } + } + break; + } + case 3: { + if (!front_trim) { + // set trim_extension + UnifiedVectorFormat sec_arg; + args.data[1].ToUnifiedFormat(args.size(), sec_arg); + if (sec_arg.validity.RowIsValid(0)) { + trim.Reinterpret(args.data[1]); + } + UnifiedVectorFormat third_arg; + args.data[2].ToUnifiedFormat(args.size(), third_arg); + if (third_arg.validity.RowIsValid(0)) { + sep.Reinterpret(args.data[2]); + } + } else { + throw InvalidInputException("Invalid number of arguments"); + } + break; + } + default: + throw InvalidInputException("Invalid number of arguments"); + } +} + +template +static void TrimPathFunction(DataChunk &args, ExpressionState &state, Vector &result) { + // set default values + Vector &path = args.data[0]; + Vector separator(string_t("default")); + Vector trim_extension(false); + ReadOptionalArgs(args, separator, trim_extension, FRONT_TRIM); + + TernaryExecutor::Execute( + path, separator, trim_extension, result, args.size(), + [&](string_t &inputs, string_t input_sep, bool trim_extension) { + auto data = inputs.GetData(); + auto input_size = inputs.GetSize(); + auto sep = GetSeparator(input_sep.GetString()); + + // find the beginning idx and the size of the result string + idx_t begin = 0; + idx_t new_size = input_size; + if (FRONT_TRIM) { // left trim + auto pos = Find(data, input_size, sep); + if (pos == 0) { // path starts with separator + pos = 1; + } + new_size = (IsIdxValid(pos, input_size)) ? pos : 0; + } else { // right trim + auto idx_last_sep = FindLast(data, input_size, sep); + if (IsIdxValid(idx_last_sep, input_size)) { + begin = idx_last_sep + 1; + } + if (trim_extension) { + auto idx_extension_sep = FindLast(data, input_size, "."); + if (begin <= idx_extension_sep && IsIdxValid(idx_extension_sep, input_size)) { + new_size = idx_extension_sep; + } + } + } + // copy the trimmed string + D_ASSERT(begin <= new_size); + auto target = StringVector::EmptyString(result, new_size - begin); + auto output = target.GetDataWriteable(); + memcpy(output, data + begin, new_size - begin); + + target.Finalize(); + return target; + }); +} + +static void ParseDirpathFunction(DataChunk &args, ExpressionState &state, Vector &result) { + // set default values + Vector &path = args.data[0]; + Vector separator(string_t("default")); + Vector trim_extension(false); + ReadOptionalArgs(args, separator, trim_extension, true); + + BinaryExecutor::Execute( + path, separator, result, args.size(), [&](string_t input_path, string_t input_sep) { + auto path = input_path.GetData(); + auto path_size = input_path.GetSize(); + auto sep = GetSeparator(input_sep.GetString()); + + auto last_sep = FindLast(path, path_size, sep); + if (last_sep == 0 && path_size == 1) { + last_sep = 1; + } + idx_t new_size = (IsIdxValid(last_sep, path_size)) ? last_sep : 0; + + auto target = StringVector::EmptyString(result, new_size); + auto output = target.GetDataWriteable(); + memcpy(output, path, new_size); + target.Finalize(); + return StringVector::AddString(result, target); + }); +} + +static void ParsePathFunction(DataChunk &args, ExpressionState &state, Vector &result) { + D_ASSERT(args.ColumnCount() == 1 || args.ColumnCount() == 2); + UnifiedVectorFormat input_data; + args.data[0].ToUnifiedFormat(args.size(), input_data); + auto inputs = UnifiedVectorFormat::GetData(input_data); + + // set the separator + string input_sep = "default"; + if (args.ColumnCount() == 2) { + UnifiedVectorFormat sep_data; + args.data[1].ToUnifiedFormat(args.size(), sep_data); + if (sep_data.validity.RowIsValid(0)) { + input_sep = UnifiedVectorFormat::GetData(sep_data)->GetString(); + } + } + const string sep = GetSeparator(input_sep); + + D_ASSERT(result.GetType().id() == LogicalTypeId::LIST); + result.SetVectorType(VectorType::FLAT_VECTOR); + ListVector::SetListSize(result, 0); + + // set up the list entries + auto list_data = FlatVector::GetData(result); + auto &child_entry = ListVector::GetEntry(result); + auto &result_mask = FlatVector::Validity(result); + idx_t total_splits = 0; + for (idx_t i = 0; i < args.size(); i++) { + auto input_idx = input_data.sel->get_index(i); + if (!input_data.validity.RowIsValid(input_idx)) { + result_mask.SetInvalid(i); + continue; + } + SplitInput split_input(result, child_entry, total_splits); + auto list_length = SplitPath(inputs[input_idx], sep, split_input); + list_data[i].length = list_length; + list_data[i].offset = total_splits; + total_splits += list_length; + } + ListVector::SetListSize(result, total_splits); + D_ASSERT(ListVector::GetListSize(result) == total_splits); + + if (args.AllConstant()) { + result.SetVectorType(VectorType::CONSTANT_VECTOR); + } +} + +ScalarFunctionSet ParseDirnameFun::GetFunctions() { + ScalarFunctionSet parse_dirname; + ScalarFunction func({LogicalType::VARCHAR}, LogicalType::VARCHAR, TrimPathFunction, nullptr, nullptr, nullptr, + nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING); + parse_dirname.AddFunction(func); + // separator options + func.arguments.emplace_back(LogicalType::VARCHAR); + parse_dirname.AddFunction(func); + return parse_dirname; +} + +ScalarFunctionSet ParseDirpathFun::GetFunctions() { + ScalarFunctionSet parse_dirpath; + ScalarFunction func({LogicalType::VARCHAR}, LogicalType::VARCHAR, ParseDirpathFunction, nullptr, nullptr, nullptr, + nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING); + parse_dirpath.AddFunction(func); + // separator options + func.arguments.emplace_back(LogicalType::VARCHAR); + parse_dirpath.AddFunction(func); + return parse_dirpath; +} + +ScalarFunctionSet ParseFilenameFun::GetFunctions() { + ScalarFunctionSet parse_filename; + parse_filename.AddFunction(ScalarFunction({LogicalType::VARCHAR}, LogicalType::VARCHAR, TrimPathFunction, + nullptr, nullptr, nullptr, nullptr, LogicalType::INVALID, + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); + parse_filename.AddFunction(ScalarFunction( + {LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::VARCHAR, TrimPathFunction, nullptr, nullptr, + nullptr, nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); + parse_filename.AddFunction(ScalarFunction( + {LogicalType::VARCHAR, LogicalType::BOOLEAN}, LogicalType::VARCHAR, TrimPathFunction, nullptr, nullptr, + nullptr, nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); + parse_filename.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::BOOLEAN, LogicalType::VARCHAR}, + LogicalType::VARCHAR, TrimPathFunction, nullptr, nullptr, nullptr, + nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING)); + return parse_filename; +} + +ScalarFunctionSet ParsePathFun::GetFunctions() { + auto varchar_list_type = LogicalType::LIST(LogicalType::VARCHAR); + ScalarFunctionSet parse_path; + ScalarFunction func({LogicalType::VARCHAR}, varchar_list_type, ParsePathFunction, nullptr, nullptr, nullptr, + nullptr, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING); + parse_path.AddFunction(func); + // separator options + func.arguments.emplace_back(LogicalType::VARCHAR); + parse_path.AddFunction(func); + return parse_path; +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/string/regexp_escape.cpp b/src/duckdb/src/core_functions/scalar/string/regexp_escape.cpp new file mode 100644 index 000000000..32517c9c8 --- /dev/null +++ b/src/duckdb/src/core_functions/scalar/string/regexp_escape.cpp @@ -0,0 +1,22 @@ +#include "duckdb/core_functions/scalar/string_functions.hpp" +#include "re2/re2.h" + +namespace duckdb { + +struct EscapeOperator { + template + static RESULT_TYPE Operation(INPUT_TYPE &input, Vector &result) { + auto escaped_pattern = RE2::QuoteMeta(input.GetString()); + return StringVector::AddString(result, escaped_pattern); + } +}; + +static void RegexpEscapeFunction(DataChunk &args, ExpressionState &state, Vector &result) { + UnaryExecutor::ExecuteString(args.data[0], result, args.size()); +} + +ScalarFunction RegexpEscapeFun::GetFunction() { + return ScalarFunction({LogicalType::VARCHAR}, LogicalType::VARCHAR, RegexpEscapeFunction); +} + +} // namespace duckdb diff --git a/src/duckdb/src/core_functions/scalar/string/string_split.cpp b/src/duckdb/src/core_functions/scalar/string/string_split.cpp index 7a41d3bd8..7a777866d 100644 --- a/src/duckdb/src/core_functions/scalar/string/string_split.cpp +++ b/src/duckdb/src/core_functions/scalar/string/string_split.cpp @@ -2,10 +2,10 @@ #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/common/types/vector.hpp" #include "duckdb/common/vector_size.hpp" -#include "duckdb/function/scalar/regexp.hpp" #include "duckdb/core_functions/scalar/string_functions.hpp" -#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/function/scalar/regexp.hpp" #include "duckdb/function/scalar/string_functions.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" namespace duckdb { @@ -24,8 +24,7 @@ struct StringSplitInput { ListVector::SetListSize(result_list, offset + list_idx); ListVector::Reserve(result_list, ListVector::GetListCapacity(result_list) * 2); } - FlatVector::GetData(result_child)[list_entry] = - StringVector::AddString(result_child, split_data, split_size); + FlatVector::GetData(result_child)[list_entry] = string_t(split_data, split_size); } }; @@ -153,6 +152,8 @@ static void StringSplitExecutor(DataChunk &args, ExpressionState &state, Vector if (args.AllConstant()) { result.SetVectorType(VectorType::CONSTANT_VECTOR); } + + StringVector::AddHeapReference(child_entry, args.data[0]); } static void StringSplitFunction(DataChunk &args, ExpressionState &state, Vector &result) { @@ -185,7 +186,7 @@ ScalarFunctionSet StringSplitRegexFun::GetFunctions() { ScalarFunctionSet regexp_split; ScalarFunction regex_fun({LogicalType::VARCHAR, LogicalType::VARCHAR}, varchar_list_type, StringSplitRegexFunction, RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING); regexp_split.AddFunction(regex_fun); // regexp options regex_fun.arguments.emplace_back(LogicalType::VARCHAR); diff --git a/src/duckdb/src/core_functions/scalar/struct/struct_insert.cpp b/src/duckdb/src/core_functions/scalar/struct/struct_insert.cpp index 3d9753f53..6a44d12a7 100644 --- a/src/duckdb/src/core_functions/scalar/struct/struct_insert.cpp +++ b/src/duckdb/src/core_functions/scalar/struct/struct_insert.cpp @@ -40,15 +40,15 @@ static unique_ptr StructInsertBind(ClientContext &context, ScalarF case_insensitive_set_t name_collision_set; if (arguments.empty()) { - throw Exception("Missing required arguments for struct_insert function."); + throw InvalidInputException("Missing required arguments for struct_insert function."); } if (LogicalTypeId::STRUCT != arguments[0]->return_type.id()) { - throw Exception("The first argument to struct_insert must be a STRUCT"); + throw InvalidInputException("The first argument to struct_insert must be a STRUCT"); } if (arguments.size() < 2) { - throw Exception("Can't insert nothing into a struct"); + throw InvalidInputException("Can't insert nothing into a struct"); } child_list_t new_struct_children; diff --git a/src/duckdb/src/core_functions/scalar/struct/struct_pack.cpp b/src/duckdb/src/core_functions/scalar/struct/struct_pack.cpp index bd6787a36..b173439f8 100644 --- a/src/duckdb/src/core_functions/scalar/struct/struct_pack.cpp +++ b/src/duckdb/src/core_functions/scalar/struct/struct_pack.cpp @@ -37,7 +37,7 @@ static unique_ptr StructPackBind(ClientContext &context, ScalarFun // collect names and deconflict, construct return type if (arguments.empty()) { - throw Exception("Can't pack nothing into a struct"); + throw InvalidInputException("Can't pack nothing into a struct"); } child_list_t struct_children; for (idx_t i = 0; i < arguments.size(); i++) { diff --git a/src/duckdb/src/execution/aggregate_hashtable.cpp b/src/duckdb/src/execution/aggregate_hashtable.cpp index d15583c2e..8a15d15e3 100644 --- a/src/duckdb/src/execution/aggregate_hashtable.cpp +++ b/src/duckdb/src/execution/aggregate_hashtable.cpp @@ -512,7 +512,7 @@ void GroupedAggregateHashTable::Combine(GroupedAggregateHashTable &other) { } } -void GroupedAggregateHashTable::Combine(TupleDataCollection &other_data) { +void GroupedAggregateHashTable::Combine(TupleDataCollection &other_data, optional_ptr> progress) { D_ASSERT(other_data.GetLayout().GetAggrWidth() == layout.GetAggrWidth()); D_ASSERT(other_data.GetLayout().GetDataWidth() == layout.GetDataWidth()); D_ASSERT(other_data.GetLayout().GetRowWidth() == layout.GetRowWidth()); @@ -523,6 +523,9 @@ void GroupedAggregateHashTable::Combine(TupleDataCollection &other_data) { FlushMoveState fm_state(other_data); RowOperationsState row_state(*aggregate_allocator); + + idx_t chunk_idx = 0; + const auto chunk_count = other_data.ChunkCount(); while (fm_state.Scan()) { FindOrCreateGroups(fm_state.groups, fm_state.hashes, fm_state.group_addresses, fm_state.new_groups_sel); RowOperations::CombineStates(row_state, layout, fm_state.scan_state.chunk_state.row_locations, @@ -531,6 +534,10 @@ void GroupedAggregateHashTable::Combine(TupleDataCollection &other_data) { RowOperations::DestroyStates(row_state, layout, fm_state.scan_state.chunk_state.row_locations, fm_state.groups.size()); } + + if (progress) { + *progress = double(++chunk_idx) / double(chunk_count); + } } Verify(); diff --git a/src/duckdb/src/execution/column_binding_resolver.cpp b/src/duckdb/src/execution/column_binding_resolver.cpp index 1bb2a74e0..ea4b2dc7c 100644 --- a/src/duckdb/src/execution/column_binding_resolver.cpp +++ b/src/duckdb/src/execution/column_binding_resolver.cpp @@ -1,16 +1,14 @@ #include "duckdb/execution/column_binding_resolver.hpp" -#include "duckdb/planner/operator/logical_comparison_join.hpp" +#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/common/to_string.hpp" +#include "duckdb/planner/expression/bound_columnref_expression.hpp" +#include "duckdb/planner/expression/bound_reference_expression.hpp" #include "duckdb/planner/operator/logical_any_join.hpp" +#include "duckdb/planner/operator/logical_comparison_join.hpp" #include "duckdb/planner/operator/logical_create_index.hpp" -#include "duckdb/planner/operator/logical_insert.hpp" #include "duckdb/planner/operator/logical_extension_operator.hpp" - -#include "duckdb/planner/expression/bound_columnref_expression.hpp" -#include "duckdb/planner/expression/bound_reference_expression.hpp" - -#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" -#include "duckdb/common/to_string.hpp" +#include "duckdb/planner/operator/logical_insert.hpp" namespace duckdb { @@ -20,8 +18,7 @@ ColumnBindingResolver::ColumnBindingResolver() { void ColumnBindingResolver::VisitOperator(LogicalOperator &op) { switch (op.type) { case LogicalOperatorType::LOGICAL_ASOF_JOIN: - case LogicalOperatorType::LOGICAL_COMPARISON_JOIN: - case LogicalOperatorType::LOGICAL_DELIM_JOIN: { + case LogicalOperatorType::LOGICAL_COMPARISON_JOIN: { // special case: comparison join auto &comp_join = op.Cast(); // first get the bindings of the LHS and resolve the LHS expressions @@ -42,6 +39,40 @@ void ColumnBindingResolver::VisitOperator(LogicalOperator &op) { bindings = op.GetColumnBindings(); return; } + case LogicalOperatorType::LOGICAL_DELIM_JOIN: { + auto &comp_join = op.Cast(); + // depending on whether the delim join has been flipped, get the appropriate bindings + if (comp_join.delim_flipped) { + VisitOperator(*comp_join.children[1]); + for (auto &cond : comp_join.conditions) { + VisitExpression(&cond.right); + } + } else { + VisitOperator(*comp_join.children[0]); + for (auto &cond : comp_join.conditions) { + VisitExpression(&cond.left); + } + } + // visit the duplicate eliminated columns + for (auto &expr : comp_join.duplicate_eliminated_columns) { + VisitExpression(&expr); + } + // now get the other side + if (comp_join.delim_flipped) { + VisitOperator(*comp_join.children[0]); + for (auto &cond : comp_join.conditions) { + VisitExpression(&cond.left); + } + } else { + VisitOperator(*comp_join.children[1]); + for (auto &cond : comp_join.conditions) { + VisitExpression(&cond.right); + } + } + // finally update the bindings with the result bindings of the join + bindings = op.GetColumnBindings(); + return; + } case LogicalOperatorType::LOGICAL_ANY_JOIN: { // ANY join, this join is different because we evaluate the expression on the bindings of BOTH join sides at // once i.e. we set the bindings first to the bindings of the entire join, and then resolve the expressions of diff --git a/src/duckdb/src/execution/expression_executor.cpp b/src/duckdb/src/execution/expression_executor.cpp index 8ee358719..25abf86f3 100644 --- a/src/duckdb/src/execution/expression_executor.cpp +++ b/src/duckdb/src/execution/expression_executor.cpp @@ -171,7 +171,8 @@ unique_ptr ExpressionExecutor::InitializeState(const Expression void ExpressionExecutor::Execute(const Expression &expr, ExpressionState *state, const SelectionVector *sel, idx_t count, Vector &result) { #ifdef DEBUG - //! The result Vector must be "clean" + // the result vector has to be used for the first time or has to be reset + // otherwise, the validity mask might contain previous (now incorrect) data if (result.GetVectorType() == VectorType::FLAT_VECTOR) { D_ASSERT(FlatVector::Validity(result).CheckAllValid(count)); } diff --git a/src/duckdb/src/execution/expression_executor/execute_between.cpp b/src/duckdb/src/execution/expression_executor/execute_between.cpp index 1fc618fe7..ca7d45f75 100644 --- a/src/duckdb/src/execution/expression_executor/execute_between.cpp +++ b/src/duckdb/src/execution/expression_executor/execute_between.cpp @@ -1,3 +1,4 @@ +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_between_expression.hpp" @@ -66,6 +67,9 @@ static idx_t BetweenLoopTypeSwitch(Vector &input, Vector &lower, Vector &upper, case PhysicalType::UINT64: return TernaryExecutor::Select(input, lower, upper, sel, count, true_sel, false_sel); + case PhysicalType::UINT128: + return TernaryExecutor::Select(input, lower, upper, sel, count, + true_sel, false_sel); case PhysicalType::FLOAT: return TernaryExecutor::Select(input, lower, upper, sel, count, true_sel, false_sel); case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/execution/expression_executor/execute_case.cpp b/src/duckdb/src/execution/expression_executor/execute_case.cpp index 16159a69b..1b5bf1f4f 100644 --- a/src/duckdb/src/execution/expression_executor/execute_case.cpp +++ b/src/duckdb/src/execution/expression_executor/execute_case.cpp @@ -1,3 +1,4 @@ +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_case_expression.hpp" @@ -170,6 +171,9 @@ void ExpressionExecutor::FillSwitch(Vector &vector, Vector &result, const Select case PhysicalType::INT128: TemplatedFillLoop(vector, result, sel, count); break; + case PhysicalType::UINT128: + TemplatedFillLoop(vector, result, sel, count); + break; case PhysicalType::FLOAT: TemplatedFillLoop(vector, result, sel, count); break; diff --git a/src/duckdb/src/execution/expression_executor/execute_comparison.cpp b/src/duckdb/src/execution/expression_executor/execute_comparison.cpp index 85b18a287..1467d6c4e 100644 --- a/src/duckdb/src/execution/expression_executor/execute_comparison.cpp +++ b/src/duckdb/src/execution/expression_executor/execute_comparison.cpp @@ -1,3 +1,4 @@ +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_comparison_expression.hpp" @@ -85,6 +86,8 @@ static idx_t TemplatedSelectOperation(Vector &left, Vector &right, const Selecti return BinaryExecutor::Select(left, right, sel, count, true_sel, false_sel); case PhysicalType::INT128: return BinaryExecutor::Select(left, right, sel, count, true_sel, false_sel); + case PhysicalType::UINT128: + return BinaryExecutor::Select(left, right, sel, count, true_sel, false_sel); case PhysicalType::FLOAT: return BinaryExecutor::Select(left, right, sel, count, true_sel, false_sel); case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/execution/index/art/art.cpp b/src/duckdb/src/execution/index/art/art.cpp index 741c3e0f8..61898a5b2 100644 --- a/src/duckdb/src/execution/index/art/art.cpp +++ b/src/duckdb/src/execution/index/art/art.cpp @@ -15,6 +15,7 @@ #include "duckdb/storage/metadata/metadata_reader.hpp" #include "duckdb/storage/table/scan_state.hpp" #include "duckdb/storage/table_io_manager.hpp" +#include "duckdb/optimizer/matcher/expression_matcher.hpp" namespace duckdb { @@ -38,7 +39,7 @@ ART::ART(const string &name, const IndexConstraintType index_constraint_type, co TableIOManager &table_io_manager, const vector> &unbound_expressions, AttachedDatabase &db, const shared_ptr, ALLOCATOR_COUNT>> &allocators_ptr, const IndexStorageInfo &info) - : Index(name, "ART", index_constraint_type, column_ids, table_io_manager, unbound_expressions, db), + : Index(name, ART::TYPE_NAME, index_constraint_type, column_ids, table_io_manager, unbound_expressions, db), allocators(allocators_ptr), owns_data(false) { // initialize all allocators @@ -81,6 +82,7 @@ ART::ART(const string &name, const IndexConstraintType index_constraint_type, co case PhysicalType::UINT16: case PhysicalType::UINT32: case PhysicalType::UINT64: + case PhysicalType::UINT128: case PhysicalType::FLOAT: case PhysicalType::DOUBLE: case PhysicalType::VARCHAR: @@ -95,8 +97,9 @@ ART::ART(const string &name, const IndexConstraintType index_constraint_type, co // Initialize Predicate Scans //===--------------------------------------------------------------------===// -unique_ptr ART::InitializeScanSinglePredicate(const Transaction &transaction, const Value &value, - const ExpressionType expression_type) { +//! Initialize a single predicate scan on the index with the given expression and column IDs +static unique_ptr InitializeScanSinglePredicate(const Transaction &transaction, const Value &value, + const ExpressionType expression_type) { // initialize point lookup auto result = make_uniq(); result->values[0] = value; @@ -104,10 +107,11 @@ unique_ptr ART::InitializeScanSinglePredicate(const Transaction return std::move(result); } -unique_ptr ART::InitializeScanTwoPredicates(const Transaction &transaction, const Value &low_value, - const ExpressionType low_expression_type, - const Value &high_value, - const ExpressionType high_expression_type) { +//! Initialize a two predicate scan on the index with the given expression and column IDs +static unique_ptr InitializeScanTwoPredicates(const Transaction &transaction, const Value &low_value, + const ExpressionType low_expression_type, + const Value &high_value, + const ExpressionType high_expression_type) { // initialize range lookup auto result = make_uniq(); result->values[0] = low_value; @@ -117,6 +121,93 @@ unique_ptr ART::InitializeScanTwoPredicates(const Transaction &t return std::move(result); } +unique_ptr ART::TryInitializeScan(const Transaction &transaction, const Expression &index_expr, + const Expression &filter_expr) { + + Value low_value, high_value, equal_value; + ExpressionType low_comparison_type = ExpressionType::INVALID, high_comparison_type = ExpressionType::INVALID; + // try to find a matching index for any of the filter expressions + + // create a matcher for a comparison with a constant + ComparisonExpressionMatcher matcher; + // match on a comparison type + matcher.expr_type = make_uniq(); + // match on a constant comparison with the indexed expression + matcher.matchers.push_back(make_uniq(const_cast(index_expr))); + matcher.matchers.push_back(make_uniq()); + + matcher.policy = SetMatcher::Policy::UNORDERED; + + vector> bindings; + if (matcher.Match(const_cast(filter_expr), bindings)) { + // range or equality comparison with constant value + // we can use our index here + // bindings[0] = the expression + // bindings[1] = the index expression + // bindings[2] = the constant + auto &comparison = bindings[0].get().Cast(); + auto constant_value = bindings[2].get().Cast().value; + auto comparison_type = comparison.type; + if (comparison.left->type == ExpressionType::VALUE_CONSTANT) { + // the expression is on the right side, we flip them around + comparison_type = FlipComparisonExpression(comparison_type); + } + if (comparison_type == ExpressionType::COMPARE_EQUAL) { + // equality value + // equality overrides any other bounds so we just break here + equal_value = constant_value; + } else if (comparison_type == ExpressionType::COMPARE_GREATERTHANOREQUALTO || + comparison_type == ExpressionType::COMPARE_GREATERTHAN) { + // greater than means this is a lower bound + low_value = constant_value; + low_comparison_type = comparison_type; + } else { + // smaller than means this is an upper bound + high_value = constant_value; + high_comparison_type = comparison_type; + } + } else if (filter_expr.type == ExpressionType::COMPARE_BETWEEN) { + // BETWEEN expression + auto &between = filter_expr.Cast(); + if (!between.input->Equals(index_expr)) { + // expression doesn't match the index expression + return nullptr; + } + if (between.lower->type != ExpressionType::VALUE_CONSTANT || + between.upper->type != ExpressionType::VALUE_CONSTANT) { + // not a constant comparison + return nullptr; + } + low_value = (between.lower->Cast()).value; + low_comparison_type = between.lower_inclusive ? ExpressionType::COMPARE_GREATERTHANOREQUALTO + : ExpressionType::COMPARE_GREATERTHAN; + high_value = (between.upper->Cast()).value; + high_comparison_type = + between.upper_inclusive ? ExpressionType::COMPARE_LESSTHANOREQUALTO : ExpressionType::COMPARE_LESSTHAN; + } + + if (!equal_value.IsNull() || !low_value.IsNull() || !high_value.IsNull()) { + // we can scan this index using this predicate: try a scan + unique_ptr index_state; + if (!equal_value.IsNull()) { + // equality predicate + index_state = InitializeScanSinglePredicate(transaction, equal_value, ExpressionType::COMPARE_EQUAL); + } else if (!low_value.IsNull() && !high_value.IsNull()) { + // two-sided predicate + index_state = InitializeScanTwoPredicates(transaction, low_value, low_comparison_type, high_value, + high_comparison_type); + } else if (!low_value.IsNull()) { + // less than predicate + index_state = InitializeScanSinglePredicate(transaction, low_value, low_comparison_type); + } else { + D_ASSERT(!high_value.IsNull()); + index_state = InitializeScanSinglePredicate(transaction, high_value, high_comparison_type); + } + return index_state; + } + return nullptr; +} + //===--------------------------------------------------------------------===// // Keys //===--------------------------------------------------------------------===// @@ -194,6 +285,9 @@ void ART::GenerateKeys(ArenaAllocator &allocator, DataChunk &input, vector(allocator, input.data[0], input.size(), keys); break; + case PhysicalType::UINT128: + TemplatedGenerateKeys(allocator, input.data[0], input.size(), keys); + break; case PhysicalType::FLOAT: TemplatedGenerateKeys(allocator, input.data[0], input.size(), keys); break; @@ -240,6 +334,9 @@ void ART::GenerateKeys(ArenaAllocator &allocator, DataChunk &input, vector(allocator, input.data[i], input.size(), keys); break; + case PhysicalType::UINT128: + ConcatenateKeys(allocator, input.data[i], input.size(), keys); + break; case PhysicalType::FLOAT: ConcatenateKeys(allocator, input.data[i], input.size(), keys); break; @@ -373,7 +470,7 @@ bool ART::ConstructFromSorted(idx_t count, vector &keys, Vector &row_ide //===--------------------------------------------------------------------===// // Insert / Verification / Constraint Checking //===--------------------------------------------------------------------===// -PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { +ErrorData ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { D_ASSERT(row_ids.GetType().InternalType() == ROW_TYPE); D_ASSERT(logical_types[0] == input.data[0].GetType()); @@ -414,8 +511,8 @@ PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { } if (failed_index != DConstants::INVALID_INDEX) { - return PreservedError(ConstraintException("PRIMARY KEY or UNIQUE constraint violated: duplicate key \"%s\"", - AppendRowError(input, failed_index))); + return ErrorData(ConstraintException("PRIMARY KEY or UNIQUE constraint violated: duplicate key \"%s\"", + AppendRowError(input, failed_index))); } #ifdef DEBUG @@ -429,10 +526,10 @@ PreservedError ART::Insert(IndexLock &lock, DataChunk &input, Vector &row_ids) { } #endif - return PreservedError(); + return ErrorData(); } -PreservedError ART::Append(IndexLock &lock, DataChunk &appended_data, Vector &row_identifiers) { +ErrorData ART::Append(IndexLock &lock, DataChunk &appended_data, Vector &row_identifiers) { DataChunk expression_result; expression_result.Initialize(Allocator::DefaultAllocator(), logical_types); @@ -661,6 +758,8 @@ static ARTKey CreateKey(ArenaAllocator &allocator, PhysicalType type, Value &val return ARTKey::CreateARTKey(allocator, value.type(), value); case PhysicalType::INT128: return ARTKey::CreateARTKey(allocator, value.type(), value); + case PhysicalType::UINT128: + return ARTKey::CreateARTKey(allocator, value.type(), value); case PhysicalType::FLOAT: return ARTKey::CreateARTKey(allocator, value.type(), value); case PhysicalType::DOUBLE: @@ -1171,4 +1270,12 @@ string ART::VerifyAndToStringInternal(const bool only_verify) { return "[empty]"; } +string ART::GetConstraintViolationMessage(VerifyExistenceType verify_type, idx_t failed_index, DataChunk &input) { + auto key_name = GenerateErrorKeyName(input, failed_index); + auto exception_msg = GenerateConstraintErrorMessage(verify_type, key_name); + return exception_msg; +} + +constexpr const char *ART::TYPE_NAME; + } // namespace duckdb diff --git a/src/duckdb/src/execution/index/art/art_key.cpp b/src/duckdb/src/execution/index/art/art_key.cpp index 5f50b4e11..ace24037e 100644 --- a/src/duckdb/src/execution/index/art/art_key.cpp +++ b/src/duckdb/src/execution/index/art/art_key.cpp @@ -14,21 +14,28 @@ ARTKey::ARTKey(ArenaAllocator &allocator, const uint32_t &len) : len(len) { template <> ARTKey ARTKey::CreateARTKey(ArenaAllocator &allocator, const LogicalType &type, string_t value) { - uint32_t len = value.GetSize() + 1; + auto string_data = const_data_ptr_cast(value.GetData()); + auto string_len = value.GetSize(); + // we need to escape \00 and \01 + idx_t escape_count = 0; + for (idx_t r = 0; r < string_len; r++) { + if (string_data[r] <= 1) { + escape_count++; + } + } + idx_t len = string_len + escape_count + 1; auto data = allocator.Allocate(len); - memcpy(data, value.GetData(), len - 1); - - // FIXME: rethink this - if (type == LogicalType::BLOB || type == LogicalType::VARCHAR) { - // indexes cannot contain BLOBs (or BLOBs cast to VARCHARs) that contain zero bytes - for (uint32_t i = 0; i < len - 1; i++) { - if (data[i] == '\0') { - throw NotImplementedException("ART indexes cannot contain BLOBs with zero bytes."); - } + // copy over the data and add in escapes + idx_t pos = 0; + for (idx_t r = 0; r < string_len; r++) { + if (string_data[r] <= 1) { + // escape + data[pos++] = '\01'; } + data[pos++] = string_data[r]; } - - data[len - 1] = '\0'; + // end with a null-terminator + data[pos] = '\0'; return ARTKey(data, len); } @@ -39,21 +46,7 @@ ARTKey ARTKey::CreateARTKey(ArenaAllocator &allocator, const LogicalType &type, template <> void ARTKey::CreateARTKey(ArenaAllocator &allocator, const LogicalType &type, ARTKey &key, string_t value) { - key.len = value.GetSize() + 1; - key.data = allocator.Allocate(key.len); - memcpy(key.data, value.GetData(), key.len - 1); - - // FIXME: rethink this - if (type == LogicalType::BLOB || type == LogicalType::VARCHAR) { - // indexes cannot contain BLOBs (or BLOBs cast to VARCHARs) that contain zero bytes - for (uint32_t i = 0; i < key.len - 1; i++) { - if (key.data[i] == '\0') { - throw NotImplementedException("ART indexes cannot contain BLOBs with zero bytes."); - } - } - } - - key.data[key.len - 1] = '\0'; + key = ARTKey::CreateARTKey(allocator, type, value); } template <> diff --git a/src/duckdb/src/execution/index/index_type_set.cpp b/src/duckdb/src/execution/index/index_type_set.cpp new file mode 100644 index 000000000..4e1dda7e6 --- /dev/null +++ b/src/duckdb/src/execution/index/index_type_set.cpp @@ -0,0 +1,32 @@ +#include "duckdb/execution/index/index_type.hpp" +#include "duckdb/execution/index/index_type_set.hpp" +#include "duckdb/execution/index/art/art.hpp" + +namespace duckdb { + +IndexTypeSet::IndexTypeSet() { + // Register the ART index type + IndexType art_index_type; + art_index_type.name = ART::TYPE_NAME; + art_index_type.create_instance = ART::Create; + RegisterIndexType(art_index_type); +} + +optional_ptr IndexTypeSet::FindByName(const string &name) { + lock_guard g(lock); + auto entry = functions.find(name); + if (entry == functions.end()) { + return nullptr; + } + return &entry->second; +} + +void IndexTypeSet::RegisterIndexType(const IndexType &index_type) { + lock_guard g(lock); + if (functions.find(index_type.name) != functions.end()) { + throw CatalogException("Index type with name \"%s\" already exists!", index_type.name.c_str()); + } + functions[index_type.name] = index_type; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/index/unknown_index.cpp b/src/duckdb/src/execution/index/unknown_index.cpp new file mode 100644 index 000000000..09b14581a --- /dev/null +++ b/src/duckdb/src/execution/index/unknown_index.cpp @@ -0,0 +1,65 @@ +#include "duckdb/execution/index/unknown_index.hpp" +#include "duckdb/parser/parsed_data/create_index_info.hpp" + +namespace duckdb { + +//------------------------------------------------------------------------------- +// Unknown index +//------------------------------------------------------------------------------- + +UnknownIndex::UnknownIndex(const string &name, const string &index_type, IndexConstraintType index_constraint_type, + const vector &column_ids, TableIOManager &table_io_manager, + const vector> &unbound_expressions, AttachedDatabase &db, + const CreateIndexInfo &create_info_p, IndexStorageInfo storage_info_p) + : Index(name, index_type, index_constraint_type, column_ids, table_io_manager, unbound_expressions, db), + create_info(create_info_p), storage_info(std::move(storage_info_p)) { +} + +string UnknownIndex::GenerateErrorMessage() const { + return StringUtil::Format( + R"(Unknown index type "%s" for index "%s". You probably need to load an extension containing this index type)", + index_type.c_str(), name.c_str()); +} + +ErrorData UnknownIndex::Append(IndexLock &, DataChunk &, Vector &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::VerifyAppend(DataChunk &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::VerifyAppend(DataChunk &, ConflictManager &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::CommitDrop(IndexLock &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::Delete(IndexLock &, DataChunk &, Vector &) { + throw NotImplementedException(GenerateErrorMessage()); +} +ErrorData UnknownIndex::Insert(IndexLock &, DataChunk &, Vector &) { + throw NotImplementedException(GenerateErrorMessage()); +} +IndexStorageInfo UnknownIndex::GetStorageInfo(bool) { + throw NotImplementedException(GenerateErrorMessage()); +} +bool UnknownIndex::MergeIndexes(IndexLock &, Index &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::Vacuum(IndexLock &) { + throw NotImplementedException(GenerateErrorMessage()); +} +idx_t UnknownIndex::GetInMemorySize(IndexLock &) { + throw NotImplementedException(GenerateErrorMessage()); +} +void UnknownIndex::CheckConstraintsForChunk(DataChunk &, ConflictManager &) { + throw NotImplementedException(GenerateErrorMessage()); +} +string UnknownIndex::VerifyAndToString(IndexLock &, bool) { + throw NotImplementedException(GenerateErrorMessage()); +} + +string UnknownIndex::GetConstraintViolationMessage(VerifyExistenceType, idx_t, DataChunk &) { + throw NotImplementedException(GenerateErrorMessage()); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/join_hashtable.cpp b/src/duckdb/src/execution/join_hashtable.cpp index eef0095fe..038393bdc 100644 --- a/src/duckdb/src/execution/join_hashtable.cpp +++ b/src/duckdb/src/execution/join_hashtable.cpp @@ -15,10 +15,10 @@ using ProbeSpill = JoinHashTable::ProbeSpill; using ProbeSpillLocalState = JoinHashTable::ProbeSpillLocalAppendState; JoinHashTable::JoinHashTable(BufferManager &buffer_manager_p, const vector &conditions_p, - vector btypes, JoinType type_p) - : buffer_manager(buffer_manager_p), conditions(conditions_p), build_types(std::move(btypes)), entry_size(0), - tuple_size(0), vfound(Value::BOOLEAN(false)), join_type(type_p), finalized(false), has_null(false), - external(false), radix_bits(4), partition_start(0), partition_end(0) { + vector btypes, JoinType type_p, const vector &output_columns_p) + : buffer_manager(buffer_manager_p), conditions(conditions_p), build_types(std::move(btypes)), + output_columns(output_columns_p), entry_size(0), tuple_size(0), vfound(Value::BOOLEAN(false)), join_type(type_p), + finalized(false), has_null(false), radix_bits(INITIAL_RADIX_BITS), partition_start(0), partition_end(0) { for (auto &condition : conditions) { D_ASSERT(condition.left->return_type == condition.right->return_type); @@ -495,7 +495,7 @@ void ScanStructure::GatherResult(Vector &result, const SelectionVector &sel_vect void ScanStructure::NextInnerJoin(DataChunk &keys, DataChunk &left, DataChunk &result) { if (ht.join_type != JoinType::RIGHT_SEMI && ht.join_type != JoinType::RIGHT_ANTI) { - D_ASSERT(result.ColumnCount() == left.ColumnCount() + ht.build_types.size()); + D_ASSERT(result.ColumnCount() == left.ColumnCount() + ht.output_columns.size()); } if (this->count == 0) { // no pointers left to chase @@ -524,10 +524,11 @@ void ScanStructure::NextInnerJoin(DataChunk &keys, DataChunk &left, DataChunk &r result.Slice(left, result_vector, result_count); // on the RHS, we need to fetch the data from the hash table - for (idx_t i = 0; i < ht.build_types.size(); i++) { + for (idx_t i = 0; i < ht.output_columns.size(); i++) { auto &vector = result.data[left.ColumnCount() + i]; - D_ASSERT(vector.GetType() == ht.build_types[i]); - GatherResult(vector, result_vector, result_count, i + ht.condition_types.size()); + const auto output_col_idx = ht.output_columns[i]; + D_ASSERT(vector.GetType() == ht.layout.GetTypes()[output_col_idx]); + GatherResult(vector, result_vector, result_count, output_col_idx); } } AdvancePointers(); @@ -773,7 +774,7 @@ void ScanStructure::NextSingleJoin(DataChunk &keys, DataChunk &input, DataChunk result.data[i].Reference(input.data[i]); } // now fetch the data from the RHS - for (idx_t i = 0; i < ht.build_types.size(); i++) { + for (idx_t i = 0; i < ht.output_columns.size(); i++) { auto &vector = result.data[input.ColumnCount() + i]; // set NULL entries for every entry that was not found for (idx_t j = 0; j < input.size(); j++) { @@ -781,8 +782,9 @@ void ScanStructure::NextSingleJoin(DataChunk &keys, DataChunk &input, DataChunk FlatVector::SetNull(vector, j, true); } } - // for the remaining values we fetch the values - GatherResult(vector, result_sel, result_sel, result_count, i + ht.condition_types.size()); + const auto output_col_idx = ht.output_columns[i]; + D_ASSERT(vector.GetType() == ht.layout.GetTypes()[output_col_idx]); + GatherResult(vector, result_sel, result_sel, result_count, output_col_idx); } result.SetCardinality(input.size()); @@ -833,7 +835,7 @@ void JoinHashTable::ScanFullOuter(JoinHTScanState &state, Vector &addresses, Dat } result.SetCardinality(found_entries); - idx_t left_column_count = result.ColumnCount() - build_types.size(); + idx_t left_column_count = result.ColumnCount() - output_columns.size(); if (join_type == JoinType::RIGHT_SEMI || join_type == JoinType::RIGHT_ANTI) { left_column_count = 0; } @@ -846,11 +848,11 @@ void JoinHashTable::ScanFullOuter(JoinHTScanState &state, Vector &addresses, Dat } // gather the values from the RHS - for (idx_t i = 0; i < build_types.size(); i++) { + for (idx_t i = 0; i < output_columns.size(); i++) { auto &vector = result.data[left_column_count + i]; - D_ASSERT(vector.GetType() == build_types[i]); - const auto col_no = condition_types.size() + i; - data_collection->Gather(addresses, sel_vector, found_entries, col_no, vector, sel_vector); + const auto output_col_idx = output_columns[i]; + D_ASSERT(vector.GetType() == layout.GetTypes()[output_col_idx]); + data_collection->Gather(addresses, sel_vector, found_entries, output_col_idx, vector, sel_vector); } } @@ -872,96 +874,88 @@ idx_t JoinHashTable::FillWithHTOffsets(JoinHTScanState &state, Vector &addresses return key_count; } -bool JoinHashTable::RequiresExternalJoin(ClientConfig &config, vector> &local_hts) { - total_count = 0; - idx_t data_size = 0; - for (auto &ht : local_hts) { - auto &local_sink_collection = ht->GetSinkCollection(); - total_count += local_sink_collection.Count(); - data_size += local_sink_collection.SizeInBytes(); +idx_t JoinHashTable::GetTotalSize(const vector &partition_sizes, const vector &partition_counts, + idx_t &max_partition_size, idx_t &max_partition_count) const { + const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); + + idx_t total_size = 0; + idx_t total_count = 0; + idx_t max_partition_ht_size = 0; + max_partition_size = 0; + max_partition_count = 0; + for (idx_t i = 0; i < num_partitions; i++) { + total_size += partition_sizes[i]; + total_count += partition_counts[i]; + + auto partition_size = partition_sizes[i] + PointerTableSize(partition_counts[i]); + if (partition_size > max_partition_ht_size) { + max_partition_ht_size = partition_size; + max_partition_size = partition_sizes[i]; + max_partition_count = partition_counts[i]; + } } if (total_count == 0) { - return false; + return 0; } - if (config.force_external) { - // Do 1 round per partition if forcing external join to test all code paths - const auto r = RadixPartitioning::NumberOfPartitions(radix_bits); - auto data_size_per_round = (data_size + r - 1) / r; - auto count_per_round = (total_count + r - 1) / r; - max_ht_size = data_size_per_round + PointerTableSize(count_per_round); - external = true; - } else { - auto ht_size = data_size + PointerTableSize(total_count); - external = ht_size > max_ht_size; - } - return external; + return total_size + PointerTableSize(total_count); } -void JoinHashTable::Unpartition() { - for (auto &partition : sink_collection->GetPartitions()) { - data_collection->Combine(*partition); +idx_t JoinHashTable::GetTotalSize(vector> &local_hts, idx_t &max_partition_size, + idx_t &max_partition_count) const { + const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); + vector partition_sizes(num_partitions, 0); + vector partition_counts(num_partitions, 0); + for (auto &ht : local_hts) { + ht->GetSinkCollection().GetSizesAndCounts(partition_sizes, partition_counts); } + + return GetTotalSize(partition_sizes, partition_counts, max_partition_size, max_partition_count); } -bool JoinHashTable::RequiresPartitioning(ClientConfig &config, vector> &local_hts) { - D_ASSERT(total_count != 0); - D_ASSERT(external); +idx_t JoinHashTable::GetRemainingSize() { + const auto num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); + auto &partitions = sink_collection->GetPartitions(); - idx_t num_partitions = RadixPartitioning::NumberOfPartitions(radix_bits); - vector partition_counts(num_partitions, 0); - vector partition_sizes(num_partitions, 0); - for (auto &ht : local_hts) { - const auto &local_partitions = ht->GetSinkCollection().GetPartitions(); - for (idx_t partition_idx = 0; partition_idx < num_partitions; partition_idx++) { - auto &local_partition = local_partitions[partition_idx]; - partition_counts[partition_idx] += local_partition->Count(); - partition_sizes[partition_idx] += local_partition->SizeInBytes(); - } + idx_t count = 0; + idx_t data_size = 0; + for (idx_t partition_idx = partition_end; partition_idx < num_partitions; partition_idx++) { + count += partitions[partition_idx]->Count(); + data_size += partitions[partition_idx]->SizeInBytes(); } - // Figure out if we can fit all single partitions in memory - idx_t max_partition_idx = 0; - idx_t max_partition_size = 0; - for (idx_t partition_idx = 0; partition_idx < num_partitions; partition_idx++) { - const auto &partition_count = partition_counts[partition_idx]; - const auto &partition_size = partition_sizes[partition_idx]; - auto partition_ht_size = partition_size + PointerTableSize(partition_count); - if (partition_ht_size > max_partition_size) { - max_partition_size = partition_ht_size; - max_partition_idx = partition_idx; - } - } + return data_size + PointerTableSize(count); +} - if (config.force_external || max_partition_size > max_ht_size) { - const auto partition_count = partition_counts[max_partition_idx]; - const auto partition_size = partition_sizes[max_partition_idx]; +void JoinHashTable::Unpartition() { + data_collection = sink_collection->GetUnpartitioned(); +} - const auto max_added_bits = RadixPartitioning::MAX_RADIX_BITS - radix_bits; - idx_t added_bits = config.force_external ? 2 : 1; - for (; added_bits < max_added_bits; added_bits++) { - double partition_multiplier = RadixPartitioning::NumberOfPartitions(added_bits); +void JoinHashTable::SetRepartitionRadixBits(vector> &local_hts, const idx_t max_ht_size, + const idx_t max_partition_size, const idx_t max_partition_count) { + D_ASSERT(max_partition_size + PointerTableSize(max_partition_count) > max_ht_size); - auto new_estimated_count = double(partition_count) / partition_multiplier; - auto new_estimated_size = double(partition_size) / partition_multiplier; - auto new_estimated_ht_size = new_estimated_size + PointerTableSize(new_estimated_count); + const auto max_added_bits = RadixPartitioning::MAX_RADIX_BITS - radix_bits; + idx_t added_bits = 1; + for (; added_bits < max_added_bits; added_bits++) { + double partition_multiplier = RadixPartitioning::NumberOfPartitions(added_bits); - if (config.force_external || new_estimated_ht_size <= double(max_ht_size) / 4) { - // Aim for an estimated partition size of max_ht_size / 4 - break; - } + auto new_estimated_size = double(max_partition_size) / partition_multiplier; + auto new_estimated_count = double(max_partition_count) / partition_multiplier; + auto new_estimated_ht_size = new_estimated_size + PointerTableSize(new_estimated_count); + + if (new_estimated_ht_size <= double(max_ht_size) / 4) { + // Aim for an estimated partition size of max_ht_size / 4 + break; } - radix_bits += added_bits; - sink_collection = - make_uniq(buffer_manager, layout, radix_bits, layout.ColumnCount() - 1); - return true; - } else { - return false; } + radix_bits += added_bits; + sink_collection = + make_uniq(buffer_manager, layout, radix_bits, layout.ColumnCount() - 1); } -void JoinHashTable::Partition(JoinHashTable &global_ht) { +void JoinHashTable::Repartition(JoinHashTable &global_ht) { auto new_sink_collection = make_uniq(buffer_manager, layout, global_ht.radix_bits, layout.ColumnCount() - 1); sink_collection->Repartition(*new_sink_collection); @@ -974,7 +968,7 @@ void JoinHashTable::Reset() { finalized = false; } -bool JoinHashTable::PrepareExternalFinalize() { +bool JoinHashTable::PrepareExternalFinalize(const idx_t max_ht_size) { if (finalized) { Reset(); } @@ -1073,18 +1067,8 @@ unique_ptr JoinHashTable::ProbeAndSpill(DataChunk &keys, TupleDat ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector &probe_types) : ht(ht), context(context), probe_types(probe_types) { - auto remaining_count = ht.GetSinkCollection().Count(); - auto remaining_data_size = ht.GetSinkCollection().SizeInBytes(); - auto remaining_ht_size = remaining_data_size + ht.PointerTableSize(remaining_count); - if (remaining_ht_size <= ht.max_ht_size) { - // No need to partition as we will only have one more probe round - partitioned = false; - } else { - // More than one probe round to go, so we need to partition - partitioned = true; - global_partitions = - make_uniq(context, probe_types, ht.radix_bits, probe_types.size() - 1); - } + global_partitions = + make_uniq(context, probe_types, ht.radix_bits, probe_types.size() - 1); column_ids.reserve(probe_types.size()); for (column_t column_id = 0; column_id < probe_types.size(); column_id++) { column_ids.emplace_back(column_id); @@ -1094,76 +1078,46 @@ ProbeSpill::ProbeSpill(JoinHashTable &ht, ClientContext &context, const vector guard(lock); - if (partitioned) { - local_partitions.emplace_back(global_partitions->CreateShared()); - local_partition_append_states.emplace_back(make_uniq()); - local_partitions.back()->InitializeAppendState(*local_partition_append_states.back()); + local_partitions.emplace_back(global_partitions->CreateShared()); + local_partition_append_states.emplace_back(make_uniq()); + local_partitions.back()->InitializeAppendState(*local_partition_append_states.back()); - result.local_partition = local_partitions.back().get(); - result.local_partition_append_state = local_partition_append_states.back().get(); - } else { - local_spill_collections.emplace_back( - make_uniq(BufferManager::GetBufferManager(context), probe_types)); - local_spill_append_states.emplace_back(make_uniq()); - local_spill_collections.back()->InitializeAppend(*local_spill_append_states.back()); - - result.local_spill_collection = local_spill_collections.back().get(); - result.local_spill_append_state = local_spill_append_states.back().get(); - } + result.local_partition = local_partitions.back().get(); + result.local_partition_append_state = local_partition_append_states.back().get(); return result; } void ProbeSpill::Append(DataChunk &chunk, ProbeSpillLocalAppendState &local_state) { - if (partitioned) { - local_state.local_partition->Append(*local_state.local_partition_append_state, chunk); - } else { - local_state.local_spill_collection->Append(*local_state.local_spill_append_state, chunk); - } + local_state.local_partition->Append(*local_state.local_partition_append_state, chunk); } void ProbeSpill::Finalize() { - if (partitioned) { - D_ASSERT(local_partitions.size() == local_partition_append_states.size()); - for (idx_t i = 0; i < local_partition_append_states.size(); i++) { - local_partitions[i]->FlushAppendState(*local_partition_append_states[i]); - } - for (auto &local_partition : local_partitions) { - global_partitions->Combine(*local_partition); - } - local_partitions.clear(); - local_partition_append_states.clear(); - } else { - if (local_spill_collections.empty()) { - global_spill_collection = - make_uniq(BufferManager::GetBufferManager(context), probe_types); - } else { - global_spill_collection = std::move(local_spill_collections[0]); - for (idx_t i = 1; i < local_spill_collections.size(); i++) { - global_spill_collection->Combine(*local_spill_collections[i]); - } - } - local_spill_collections.clear(); - local_spill_append_states.clear(); + D_ASSERT(local_partitions.size() == local_partition_append_states.size()); + for (idx_t i = 0; i < local_partition_append_states.size(); i++) { + local_partitions[i]->FlushAppendState(*local_partition_append_states[i]); + } + for (auto &local_partition : local_partitions) { + global_partitions->Combine(*local_partition); } + local_partitions.clear(); + local_partition_append_states.clear(); } void ProbeSpill::PrepareNextProbe() { - if (partitioned) { - auto &partitions = global_partitions->GetPartitions(); - if (partitions.empty() || ht.partition_start == partitions.size()) { - // Can't probe, just make an empty one - global_spill_collection = - make_uniq(BufferManager::GetBufferManager(context), probe_types); - } else { - // Move specific partitions to the global spill collection - global_spill_collection = std::move(partitions[ht.partition_start]); - for (idx_t i = ht.partition_start + 1; i < ht.partition_end; i++) { - auto &partition = partitions[i]; - if (global_spill_collection->Count() == 0) { - global_spill_collection = std::move(partition); - } else { - global_spill_collection->Combine(*partition); - } + auto &partitions = global_partitions->GetPartitions(); + if (partitions.empty() || ht.partition_start == partitions.size()) { + // Can't probe, just make an empty one + global_spill_collection = + make_uniq(BufferManager::GetBufferManager(context), probe_types); + } else { + // Move specific partitions to the global spill collection + global_spill_collection = std::move(partitions[ht.partition_start]); + for (idx_t i = ht.partition_start + 1; i < ht.partition_end; i++) { + auto &partition = partitions[i]; + if (global_spill_collection->Count() == 0) { + global_spill_collection = std::move(partition); + } else { + global_spill_collection->Combine(*partition); } } } diff --git a/src/duckdb/src/execution/nested_loop_join/nested_loop_join_inner.cpp b/src/duckdb/src/execution/nested_loop_join/nested_loop_join_inner.cpp index f64943fa7..583c5946f 100644 --- a/src/duckdb/src/execution/nested_loop_join/nested_loop_join_inner.cpp +++ b/src/duckdb/src/execution/nested_loop_join/nested_loop_join_inner.cpp @@ -1,4 +1,5 @@ #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/execution/nested_loop_join.hpp" namespace duckdb { @@ -108,6 +109,9 @@ static idx_t NestedLoopJoinTypeSwitch(Vector &left, Vector &right, idx_t left_si case PhysicalType::INT128: return NLTYPE::template Operation(left, right, left_size, right_size, lpos, rpos, lvector, rvector, current_match_count); + case PhysicalType::UINT128: + return NLTYPE::template Operation(left, right, left_size, right_size, lpos, rpos, lvector, + rvector, current_match_count); case PhysicalType::FLOAT: return NLTYPE::template Operation(left, right, left_size, right_size, lpos, rpos, lvector, rvector, current_match_count); 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 67f047116..15977ac18 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 @@ -1,4 +1,5 @@ #include "duckdb/common/operator/comparison_operators.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/nested_loop_join.hpp" @@ -103,6 +104,8 @@ static void MarkJoinSwitch(Vector &left, Vector &right, idx_t lcount, idx_t rcou return TemplatedMarkJoin(left, right, lcount, rcount, found_match); case PhysicalType::UINT64: return TemplatedMarkJoin(left, right, lcount, rcount, found_match); + case PhysicalType::UINT128: + return TemplatedMarkJoin(left, right, lcount, rcount, found_match); case PhysicalType::FLOAT: return TemplatedMarkJoin(left, right, lcount, rcount, found_match); case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/execution/operator/aggregate/aggregate_object.cpp b/src/duckdb/src/execution/operator/aggregate/aggregate_object.cpp index b2ef94782..43c6dce93 100644 --- a/src/duckdb/src/execution/operator/aggregate/aggregate_object.cpp +++ b/src/duckdb/src/execution/operator/aggregate/aggregate_object.cpp @@ -22,7 +22,8 @@ AggregateObject::AggregateObject(BoundAggregateExpression *aggr) AggregateObject::AggregateObject(BoundWindowExpression &window) : AggregateObject(*window.aggregate, window.bind_info.get(), window.children.size(), - AlignValue(window.aggregate->state_size()), AggregateType::NON_DISTINCT, + AlignValue(window.aggregate->state_size()), + window.distinct ? AggregateType::DISTINCT : AggregateType::NON_DISTINCT, window.return_type.InternalType(), window.filter_expr.get()) { } diff --git a/src/duckdb/src/execution/operator/aggregate/physical_hash_aggregate.cpp b/src/duckdb/src/execution/operator/aggregate/physical_hash_aggregate.cpp index 8561c5fb1..3b62fc69f 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_hash_aggregate.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_hash_aggregate.cpp @@ -521,7 +521,7 @@ class HashAggregateDistinctFinalizeEvent : public BasePipelineEvent { void FinishEvent() override; private: - void CreateGlobalSources(); + idx_t CreateGlobalSources(); private: ClientContext &context; @@ -556,9 +556,7 @@ class HashAggregateDistinctFinalizeTask : public ExecutorTask { }; void HashAggregateDistinctFinalizeEvent::Schedule() { - CreateGlobalSources(); - - const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + const auto n_threads = CreateGlobalSources(); vector> tasks; for (idx_t i = 0; i < n_threads; i++) { tasks.push_back(make_uniq(*pipeline, shared_from_this(), op, gstate)); @@ -566,11 +564,14 @@ void HashAggregateDistinctFinalizeEvent::Schedule() { SetTasks(std::move(tasks)); } -void HashAggregateDistinctFinalizeEvent::CreateGlobalSources() { +idx_t HashAggregateDistinctFinalizeEvent::CreateGlobalSources() { auto &aggregates = op.grouped_aggregate_data.aggregates; global_source_states.reserve(op.groupings.size()); + + idx_t n_threads = 0; for (idx_t grouping_idx = 0; grouping_idx < op.groupings.size(); grouping_idx++) { auto &grouping = op.groupings[grouping_idx]; + auto &distinct_state = *gstate.grouping_states[grouping_idx].distinct_state; auto &distinct_data = *grouping.distinct_data; vector> aggregate_sources; @@ -587,10 +588,13 @@ void HashAggregateDistinctFinalizeEvent::CreateGlobalSources() { auto table_idx = distinct_data.info.table_map.at(agg_idx); auto &radix_table_p = distinct_data.radix_tables[table_idx]; + n_threads += radix_table_p->MaxThreads(*distinct_state.radix_states[table_idx]); aggregate_sources.push_back(radix_table_p->GetGlobalSourceState(context)); } global_source_states.push_back(std::move(aggregate_sources)); } + + return MaxValue(n_threads, 1); } void HashAggregateDistinctFinalizeEvent::FinishEvent() { @@ -782,13 +786,13 @@ class HashAggregateGlobalSourceState : public GlobalSourceState { } auto &ht_state = op.sink_state->Cast(); - idx_t partitions = 0; + idx_t threads = 0; for (size_t sidx = 0; sidx < op.groupings.size(); ++sidx) { auto &grouping = op.groupings[sidx]; auto &grouping_gstate = ht_state.grouping_states[sidx]; - partitions += grouping.table_data.NumberOfPartitions(*grouping_gstate.table_state); + threads += grouping.table_data.MaxThreads(*grouping_gstate.table_state); } - return MaxValue(1, partitions); + return MaxValue(1, threads); } }; @@ -850,6 +854,17 @@ SourceResultType PhysicalHashAggregate::GetData(ExecutionContext &context, DataC return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } +double PhysicalHashAggregate::GetProgress(ClientContext &context, GlobalSourceState &gstate_p) const { + auto &sink_gstate = sink_state->Cast(); + auto &gstate = gstate_p.Cast(); + double total_progress = 0; + for (idx_t radix_idx = 0; radix_idx < groupings.size(); radix_idx++) { + total_progress += groupings[radix_idx].table_data.GetProgress( + context, *sink_gstate.grouping_states[radix_idx].table_state, *gstate.radix_states[radix_idx]); + } + return total_progress / double(groupings.size()); +} + string PhysicalHashAggregate::ParamsToString() const { string result; auto &groups = grouped_aggregate_data.groups; diff --git a/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp b/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp index 8db0ede31..c72709ba0 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_ungrouped_aggregate.cpp @@ -85,29 +85,41 @@ struct AggregateState { class UngroupedAggregateGlobalSinkState : public GlobalSinkState { public: UngroupedAggregateGlobalSinkState(const PhysicalUngroupedAggregate &op, ClientContext &client) - : state(op.aggregates), finished(false), allocator(BufferAllocator::Get(client)) { + : state(op.aggregates), finished(false), client_allocator(BufferAllocator::Get(client)), + allocator(client_allocator) { if (op.distinct_data) { distinct_state = make_uniq(*op.distinct_data, client); } } + //! Create an ArenaAllocator with cross-thread lifetime + ArenaAllocator &CreateAllocator() const { + lock_guard glock(lock); + stored_allocators.emplace_back(make_uniq(client_allocator)); + return *stored_allocators.back(); + } + //! The lock for updating the global aggregate state - mutex lock; + mutable mutex lock; //! The global aggregate state AggregateState state; //! Whether or not the aggregate is finished bool finished; //! The data related to the distinct aggregates (if there are any) unique_ptr distinct_state; + //! Client base allocator + Allocator &client_allocator; //! Global arena allocator ArenaAllocator allocator; + //! Allocator pool + mutable vector> stored_allocators; }; class UngroupedAggregateLocalSinkState : public LocalSinkState { public: UngroupedAggregateLocalSinkState(const PhysicalUngroupedAggregate &op, const vector &child_types, - GlobalSinkState &gstate_p, ExecutionContext &context) - : allocator(BufferAllocator::Get(context.client)), state(op.aggregates), child_executor(context.client), + UngroupedAggregateGlobalSinkState &gstate_p, ExecutionContext &context) + : allocator(gstate_p.CreateAllocator()), state(op.aggregates), child_executor(context.client), aggregate_input_chunk(), filter_set() { auto &gstate = gstate_p.Cast(); @@ -133,7 +145,7 @@ class UngroupedAggregateLocalSinkState : public LocalSinkState { } //! Local arena allocator - ArenaAllocator allocator; + ArenaAllocator &allocator; //! The local aggregate state AggregateState state; //! The executor @@ -192,7 +204,7 @@ unique_ptr PhysicalUngroupedAggregate::GetGlobalSinkState(Clien unique_ptr PhysicalUngroupedAggregate::GetLocalSinkState(ExecutionContext &context) const { D_ASSERT(sink_state); - auto &gstate = *sink_state; + auto &gstate = sink_state->Cast(); return make_uniq(*this, children[0]->GetTypes(), gstate, context); } @@ -342,13 +354,13 @@ SinkCombineResultType PhysicalUngroupedAggregate::Combine(ExecutionContext &cont Vector source_state(Value::POINTER(CastPointerToValue(lstate.state.aggregates[aggr_idx].get()))); Vector dest_state(Value::POINTER(CastPointerToValue(gstate.state.aggregates[aggr_idx].get()))); - AggregateInputData aggr_input_data(aggregate.bind_info.get(), gstate.allocator); + AggregateInputData aggr_input_data(aggregate.bind_info.get(), gstate.allocator, + AggregateCombineType::ALLOW_DESTRUCTIVE); aggregate.function.combine(source_state, dest_state, aggr_input_data, 1); #ifdef DEBUG gstate.state.counts[aggr_idx] += lstate.state.counts[aggr_idx]; #endif } - lstate.allocator.Destroy(); auto &client_profiler = QueryProfiler::Get(context.client); context.thread.profiler.Flush(*this, lstate.child_executor, "child_executor", 0); @@ -391,7 +403,7 @@ class UngroupedDistinctAggregateFinalizeTask : public ExecutorTask { const PhysicalUngroupedAggregate &op, UngroupedAggregateGlobalSinkState &state_p) : ExecutorTask(executor), event(std::move(event_p)), op(op), gstate(state_p), - allocator(BufferAllocator::Get(executor.context)) { + allocator(gstate.CreateAllocator()) { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override; @@ -405,7 +417,7 @@ class UngroupedDistinctAggregateFinalizeTask : public ExecutorTask { const PhysicalUngroupedAggregate &op; UngroupedAggregateGlobalSinkState &gstate; - ArenaAllocator allocator; + ArenaAllocator &allocator; }; void UngroupedDistinctAggregateFinalizeEvent::Schedule() { @@ -413,6 +425,7 @@ void UngroupedDistinctAggregateFinalizeEvent::Schedule() { auto &aggregates = op.aggregates; auto &distinct_data = *op.distinct_data; + idx_t n_threads = 0; idx_t payload_idx = 0; idx_t next_payload_idx = 0; for (idx_t agg_idx = 0; agg_idx < aggregates.size(); agg_idx++) { @@ -432,10 +445,11 @@ void UngroupedDistinctAggregateFinalizeEvent::Schedule() { // Create global state for scanning auto table_idx = distinct_data.info.table_map.at(agg_idx); auto &radix_table_p = *distinct_data.radix_tables[table_idx]; + n_threads += radix_table_p.MaxThreads(*gstate.distinct_state->radix_states[table_idx]); global_source_states.push_back(radix_table_p.GetGlobalSourceState(context)); } + n_threads = MaxValue(n_threads, 1); - const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); vector> tasks; for (idx_t i = 0; i < n_threads; i++) { tasks.push_back( @@ -537,7 +551,8 @@ void UngroupedDistinctAggregateFinalizeTask::AggregateDistinct() { } auto &aggregate = aggregates[agg_idx]->Cast(); - AggregateInputData aggr_input_data(aggregate.bind_info.get(), allocator); + AggregateInputData aggr_input_data(aggregate.bind_info.get(), allocator, + AggregateCombineType::ALLOW_DESTRUCTIVE); Vector state_vec(Value::POINTER(CastPointerToValue(state.aggregates[agg_idx].get()))); Vector combined_vec(Value::POINTER(CastPointerToValue(gstate.state.aggregates[agg_idx].get()))); diff --git a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp index d1981e01f..bcfe0a56b 100644 --- a/src/duckdb/src/execution/operator/aggregate/physical_window.cpp +++ b/src/duckdb/src/execution/operator/aggregate/physical_window.cpp @@ -8,9 +8,10 @@ #include "duckdb/common/radix_partitioning.hpp" #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/sort/partition_state.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/common/types/column/column_data_consumer.hpp" #include "duckdb/common/types/row/row_data_collection_scanner.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/windows_undefs.hpp" #include "duckdb/execution/expression_executor.hpp" diff --git a/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp deleted file mode 100644 index c27255529..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/base_csv_reader.cpp +++ /dev/null @@ -1,597 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" - -#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" -#include "duckdb/common/file_system.hpp" -#include "duckdb/common/string_util.hpp" -#include "duckdb/common/to_string.hpp" -#include "duckdb/common/types/cast_helpers.hpp" -#include "duckdb/common/operator/cast_operators.hpp" -#include "duckdb/common/operator/decimal_cast_operators.hpp" -#include "duckdb/common/vector_operations/unary_executor.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" -#include "duckdb/function/scalar/strftime_format.hpp" -#include "duckdb/main/appender.hpp" -#include "duckdb/main/database.hpp" -#include "duckdb/parser/column_definition.hpp" -#include "duckdb/storage/data_table.hpp" -#include "utf8proc_wrapper.hpp" -#include "utf8proc.hpp" -#include "duckdb/parser/keyword_helper.hpp" -#include "duckdb/main/error_manager.hpp" -#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" -#include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" -#include "duckdb/main/client_data.hpp" -#include -#include -#include -#include - -namespace duckdb { - -string BaseCSVReader::GetLineNumberStr(idx_t line_error, bool is_line_estimated, idx_t buffer_idx) { - // If an error happens during auto-detect it is an estimated line - string estimated = (is_line_estimated ? string(" (estimated)") : string("")); - return to_string(GetLineError(line_error, buffer_idx)) + estimated; -} - -BaseCSVReader::BaseCSVReader(ClientContext &context_p, CSVReaderOptions options_p, - const vector &requested_types) - : context(context_p), fs(FileSystem::GetFileSystem(context)), allocator(BufferAllocator::Get(context)), - options(std::move(options_p)) { -} - -BaseCSVReader::~BaseCSVReader() { -} - -unique_ptr BaseCSVReader::OpenCSV(ClientContext &context, const CSVReaderOptions &options_p) { - return CSVFileHandle::OpenFile(FileSystem::GetFileSystem(context), BufferAllocator::Get(context), - options_p.file_path, options_p.compression); -} - -void BaseCSVReader::InitParseChunk(idx_t num_cols) { - // adapt not null info - if (options.force_not_null.size() != num_cols) { - options.force_not_null.resize(num_cols, false); - } - if (num_cols == parse_chunk.ColumnCount()) { - parse_chunk.Reset(); - } else { - parse_chunk.Destroy(); - - // initialize the parse_chunk with a set of VARCHAR types - vector varchar_types(num_cols, LogicalType::VARCHAR); - parse_chunk.Initialize(allocator, varchar_types); - } -} - -void BaseCSVReader::InitializeProjection() { - for (idx_t i = 0; i < GetTypes().size(); i++) { - reader_data.column_ids.push_back(i); - reader_data.column_mapping.push_back(i); - } -} - -template -static bool TemplatedTryCastDateVector(map> &options, Vector &input_vector, - Vector &result_vector, idx_t count, string &error_message, idx_t &line_error) { - D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); - bool all_converted = true; - idx_t cur_line = 0; - UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { - T result; - if (!OP::Operation(options, input, result, error_message)) { - FlatVector::SetNull(result_vector, cur_line, true); - line_error = cur_line; - all_converted = false; - } - cur_line++; - return result; - }); - return all_converted; -} - -struct TryCastDateOperator { - static bool Operation(map> &options, string_t input, date_t &result, - string &error_message) { - return options[LogicalTypeId::DATE].GetValue().TryParseDate(input, result, error_message); - } -}; - -struct TryCastTimestampOperator { - static bool Operation(map> &options, string_t input, timestamp_t &result, - string &error_message) { - return options[LogicalTypeId::TIMESTAMP].GetValue().TryParseTimestamp(input, result, error_message); - } -}; - -bool BaseCSVReader::TryCastDateVector(map> &options, Vector &input_vector, - Vector &result_vector, idx_t count, string &error_message, idx_t &line_error) { - return TemplatedTryCastDateVector(options, input_vector, result_vector, count, - error_message, line_error); -} - -bool BaseCSVReader::TryCastTimestampVector(map> &options, Vector &input_vector, - Vector &result_vector, idx_t count, string &error_message) { - idx_t line_error; - return TemplatedTryCastDateVector(options, input_vector, result_vector, - count, error_message, line_error); -} - -void BaseCSVReader::VerifyLineLength(idx_t line_size, idx_t buffer_idx) { - if (line_size > options.maximum_line_size) { - throw InvalidInputException( - "Error in file \"%s\" on line %s: Maximum line size of %llu bytes exceeded!", options.file_path, - GetLineNumberStr(parse_chunk.size(), linenr_estimated, buffer_idx).c_str(), options.maximum_line_size); - } -} - -template -bool TemplatedTryCastFloatingVector(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, idx_t count, - string &error_message, idx_t &line_error) { - D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); - bool all_converted = true; - idx_t row = 0; - UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { - T result; - if (!OP::Operation(input, result, &error_message)) { - line_error = row; - all_converted = false; - } else { - row++; - } - return result; - }); - return all_converted; -} - -template -bool TemplatedTryCastDecimalVector(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, idx_t count, - string &error_message, uint8_t width, uint8_t scale) { - D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); - bool all_converted = true; - UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { - T result; - if (!OP::Operation(input, result, &error_message, width, scale)) { - all_converted = false; - } - return result; - }); - return all_converted; -} - -void BaseCSVReader::AddValue(string_t str_val, idx_t &column, vector &escape_positions, bool has_quotes, - idx_t buffer_idx) { - auto length = str_val.GetSize(); - if (length == 0 && column == 0) { - row_empty = true; - } else { - row_empty = false; - } - if (!return_types.empty() && column == return_types.size() && length == 0) { - // skip a single trailing delimiter in last column - return; - } - if (column >= return_types.size()) { - if (options.ignore_errors) { - error_column_overflow = true; - return; - } else { - throw InvalidInputException( - "Error in file \"%s\", on line %s: expected %lld values per row, but got more. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated, buffer_idx).c_str(), return_types.size(), - options.ToString()); - } - } - - // insert the line number into the chunk - idx_t row_entry = parse_chunk.size(); - - // test against null string, but only if the value was not quoted - if ((!(has_quotes && !options.allow_quoted_nulls) || return_types[column].id() != LogicalTypeId::VARCHAR) && - !options.force_not_null[column] && Equals::Operation(str_val, string_t(options.null_str))) { - FlatVector::SetNull(parse_chunk.data[column], row_entry, true); - } else { - auto &v = parse_chunk.data[column]; - auto parse_data = FlatVector::GetData(v); - if (!escape_positions.empty()) { - // remove escape characters (if any) - string old_val = str_val.GetString(); - string new_val = ""; - idx_t prev_pos = 0; - for (idx_t i = 0; i < escape_positions.size(); i++) { - idx_t next_pos = escape_positions[i]; - new_val += old_val.substr(prev_pos, next_pos - prev_pos); - prev_pos = ++next_pos; - } - new_val += old_val.substr(prev_pos, old_val.size() - prev_pos); - escape_positions.clear(); - parse_data[row_entry] = StringVector::AddStringOrBlob(v, string_t(new_val)); - } else { - parse_data[row_entry] = str_val; - } - } - - // move to the next column - column++; -} - -bool BaseCSVReader::AddRow(DataChunk &insert_chunk, idx_t &column, string &error_message, idx_t buffer_idx) { - linenr++; - - if (row_empty) { - row_empty = false; - if (return_types.size() != 1) { - if (mode == ParserMode::PARSING) { - FlatVector::SetNull(parse_chunk.data[0], parse_chunk.size(), false); - } - column = 0; - return false; - } - } - - // Error forwarded by 'ignore_errors' - originally encountered in 'AddValue' - if (error_column_overflow) { - D_ASSERT(options.ignore_errors); - error_column_overflow = false; - column = 0; - return false; - } - - if (column < return_types.size()) { - if (options.null_padding) { - for (; column < return_types.size(); column++) { - FlatVector::SetNull(parse_chunk.data[column], parse_chunk.size(), true); - } - } else if (options.ignore_errors) { - column = 0; - return false; - } else { - if (mode == ParserMode::SNIFFING_DATATYPES) { - error_message = "Error when adding line"; - return false; - } else { - throw InvalidInputException( - "Error in file \"%s\" on line %s: expected %lld values per row, but got %d.\nParser options:\n%s", - options.file_path, GetLineNumberStr(linenr, linenr_estimated, buffer_idx).c_str(), - return_types.size(), column, options.ToString()); - } - } - } - - parse_chunk.SetCardinality(parse_chunk.size() + 1); - - if (mode == ParserMode::PARSING_HEADER) { - return true; - } - - if (mode == ParserMode::SNIFFING_DATATYPES) { - return true; - } - - if (mode == ParserMode::PARSING && parse_chunk.size() == STANDARD_VECTOR_SIZE) { - Flush(insert_chunk, buffer_idx); - return true; - } - - column = 0; - return false; -} - -void BaseCSVReader::VerifyUTF8(idx_t col_idx, idx_t row_idx, DataChunk &chunk, int64_t offset) { - D_ASSERT(col_idx < chunk.data.size()); - D_ASSERT(row_idx < chunk.size()); - auto &v = chunk.data[col_idx]; - if (FlatVector::IsNull(v, row_idx)) { - return; - } - - auto parse_data = FlatVector::GetData(chunk.data[col_idx]); - auto s = parse_data[row_idx]; - auto utf_type = Utf8Proc::Analyze(s.GetData(), s.GetSize()); - if (utf_type == UnicodeType::INVALID) { - string col_name = to_string(col_idx); - if (col_idx < names.size()) { - col_name = "\"" + names[col_idx] + "\""; - } - int64_t error_line = linenr - (chunk.size() - row_idx) + 1 + offset; - D_ASSERT(error_line >= 0); - throw InvalidInputException("Error in file \"%s\" at line %llu in column \"%s\": " - "%s. Parser options:\n%s", - options.file_path, error_line, col_name, - ErrorManager::InvalidUnicodeError(s.GetString(), "CSV file"), options.ToString()); - } -} - -void BaseCSVReader::VerifyUTF8(idx_t col_idx) { - D_ASSERT(col_idx < parse_chunk.data.size()); - for (idx_t i = 0; i < parse_chunk.size(); i++) { - VerifyUTF8(col_idx, i, parse_chunk); - } -} - -bool TryCastDecimalVectorCommaSeparated(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, - idx_t count, string &error_message, const LogicalType &result_type) { - auto width = DecimalType::GetWidth(result_type); - auto scale = DecimalType::GetScale(result_type); - switch (result_type.InternalType()) { - case PhysicalType::INT16: - return TemplatedTryCastDecimalVector( - options, input_vector, result_vector, count, error_message, width, scale); - case PhysicalType::INT32: - return TemplatedTryCastDecimalVector( - options, input_vector, result_vector, count, error_message, width, scale); - case PhysicalType::INT64: - return TemplatedTryCastDecimalVector( - options, input_vector, result_vector, count, error_message, width, scale); - case PhysicalType::INT128: - return TemplatedTryCastDecimalVector( - options, input_vector, result_vector, count, error_message, width, scale); - default: - throw InternalException("Unimplemented physical type for decimal"); - } -} - -bool TryCastFloatingVectorCommaSeparated(CSVReaderOptions &options, Vector &input_vector, Vector &result_vector, - idx_t count, string &error_message, const LogicalType &result_type, - idx_t &line_error) { - switch (result_type.InternalType()) { - case PhysicalType::DOUBLE: - return TemplatedTryCastFloatingVector( - options, input_vector, result_vector, count, error_message, line_error); - case PhysicalType::FLOAT: - return TemplatedTryCastFloatingVector( - options, input_vector, result_vector, count, error_message, line_error); - default: - throw InternalException("Unimplemented physical type for floating"); - } -} - -// Location of erroneous value in the current parse chunk -struct ErrorLocation { - idx_t row_idx; - idx_t col_idx; - idx_t row_line; - - ErrorLocation(idx_t row_idx, idx_t col_idx, idx_t row_line) - : row_idx(row_idx), col_idx(col_idx), row_line(row_line) { - } -}; - -bool BaseCSVReader::Flush(DataChunk &insert_chunk, idx_t buffer_idx, bool try_add_line) { - if (parse_chunk.size() == 0) { - return true; - } - - bool conversion_error_ignored = false; - - // convert the columns in the parsed chunk to the types of the table - insert_chunk.SetCardinality(parse_chunk); - if (reader_data.column_ids.empty() && !reader_data.empty_columns) { - throw InternalException("BaseCSVReader::Flush called on a CSV reader that was not correctly initialized. Call " - "MultiFileReader::InitializeReader or InitializeProjection"); - } - D_ASSERT(reader_data.column_ids.size() == reader_data.column_mapping.size()); - for (idx_t c = 0; c < reader_data.column_ids.size(); c++) { - auto col_idx = reader_data.column_ids[c]; - auto result_idx = reader_data.column_mapping[c]; - auto &parse_vector = parse_chunk.data[col_idx]; - auto &result_vector = insert_chunk.data[result_idx]; - auto &type = result_vector.GetType(); - if (type.id() == LogicalTypeId::VARCHAR) { - // target type is varchar: no need to convert - // just test that all strings are valid utf-8 strings - VerifyUTF8(col_idx); - // reinterpret rather than reference so we can deal with user-defined types - result_vector.Reinterpret(parse_vector); - } else { - string error_message; - bool success; - idx_t line_error = 0; - bool target_type_not_varchar = false; - if (!options.dialect_options.date_format[LogicalTypeId::DATE].GetValue().Empty() && - type.id() == LogicalTypeId::DATE) { - // use the date format to cast the chunk - success = TryCastDateVector(options.dialect_options.date_format, parse_vector, result_vector, - parse_chunk.size(), error_message, line_error); - } else if (!options.dialect_options.date_format[LogicalTypeId::TIMESTAMP].GetValue().Empty() && - type.id() == LogicalTypeId::TIMESTAMP) { - // use the date format to cast the chunk - success = TryCastTimestampVector(options.dialect_options.date_format, parse_vector, result_vector, - parse_chunk.size(), error_message); - } else if (options.decimal_separator != "." && - (type.id() == LogicalTypeId::FLOAT || type.id() == LogicalTypeId::DOUBLE)) { - success = TryCastFloatingVectorCommaSeparated(options, parse_vector, result_vector, parse_chunk.size(), - error_message, type, line_error); - } else if (options.decimal_separator != "." && type.id() == LogicalTypeId::DECIMAL) { - success = TryCastDecimalVectorCommaSeparated(options, parse_vector, result_vector, parse_chunk.size(), - error_message, type); - } else { - // target type is not varchar: perform a cast - target_type_not_varchar = true; - success = - VectorOperations::TryCast(context, parse_vector, result_vector, parse_chunk.size(), &error_message); - } - if (success) { - continue; - } - if (try_add_line) { - return false; - } - - string col_name = to_string(col_idx); - if (col_idx < names.size()) { - col_name = "\"" + names[col_idx] + "\""; - } - - // figure out the exact line number - if (target_type_not_varchar) { - UnifiedVectorFormat inserted_column_data; - result_vector.ToUnifiedFormat(parse_chunk.size(), inserted_column_data); - for (; line_error < parse_chunk.size(); line_error++) { - if (!inserted_column_data.validity.RowIsValid(line_error) && - !FlatVector::IsNull(parse_vector, line_error)) { - break; - } - } - } - - // The line_error must be summed with linenr (All lines emmited from this batch) - // But subtracted from the parse_chunk - D_ASSERT(line_error + linenr >= parse_chunk.size()); - line_error += linenr; - line_error -= parse_chunk.size(); - - auto error_line = GetLineError(line_error, buffer_idx); - - if (options.ignore_errors) { - conversion_error_ignored = true; - - } else if (options.auto_detect) { - throw InvalidInputException("%s in column %s, at line %llu.\n\nParser " - "options:\n%s.\n\nConsider either increasing the sample size " - "(SAMPLE_SIZE=X [X rows] or SAMPLE_SIZE=-1 [all rows]), " - "or skipping column conversion (ALL_VARCHAR=1)", - error_message, col_name, error_line, options.ToString()); - } else { - throw InvalidInputException("%s at line %llu in column %s. Parser options:\n%s ", error_message, - error_line, col_name, options.ToString()); - } - } - } - if (conversion_error_ignored) { - D_ASSERT(options.ignore_errors); - - SelectionVector succesful_rows(parse_chunk.size()); - idx_t sel_size = 0; - - // Keep track of failed cells - vector failed_cells; - - for (idx_t row_idx = 0; row_idx < parse_chunk.size(); row_idx++) { - - auto global_row_idx = row_idx + linenr - parse_chunk.size(); - auto row_line = GetLineError(global_row_idx, buffer_idx, false); - - bool row_failed = false; - for (idx_t c = 0; c < reader_data.column_ids.size(); c++) { - auto col_idx = reader_data.column_ids[c]; - auto result_idx = reader_data.column_mapping[c]; - - auto &parse_vector = parse_chunk.data[col_idx]; - auto &result_vector = insert_chunk.data[result_idx]; - - bool was_already_null = FlatVector::IsNull(parse_vector, row_idx); - if (!was_already_null && FlatVector::IsNull(result_vector, row_idx)) { - Increment(buffer_idx); - auto bla = GetLineError(global_row_idx, buffer_idx, false); - row_idx += bla; - row_idx -= bla; - row_failed = true; - failed_cells.emplace_back(row_idx, col_idx, row_line); - } - } - if (!row_failed) { - succesful_rows.set_index(sel_size++, row_idx); - } - } - - // Now do a second pass to produce the reject table entries - if (!failed_cells.empty() && !options.rejects_table_name.empty()) { - auto limit = options.rejects_limit; - - auto rejects = CSVRejectsTable::GetOrCreate(context, options.rejects_table_name); - lock_guard lock(rejects->write_lock); - - // short circuit if we already have too many rejects - if (limit == 0 || rejects->count < limit) { - auto &table = rejects->GetTable(context); - InternalAppender appender(context, table); - auto file_name = GetFileName(); - - for (auto &cell : failed_cells) { - if (limit != 0 && rejects->count >= limit) { - break; - } - rejects->count++; - - auto row_idx = cell.row_idx; - auto col_idx = cell.col_idx; - auto row_line = cell.row_line; - - auto col_name = to_string(col_idx); - if (col_idx < names.size()) { - col_name = "\"" + names[col_idx] + "\""; - } - - auto &parse_vector = parse_chunk.data[col_idx]; - auto parsed_str = FlatVector::GetData(parse_vector)[row_idx]; - auto &type = insert_chunk.data[col_idx].GetType(); - auto row_error_msg = StringUtil::Format("Could not convert string '%s' to '%s'", - parsed_str.GetString(), type.ToString()); - - // Add the row to the rejects table - appender.BeginRow(); - appender.Append(string_t(file_name)); - appender.Append(row_line); - appender.Append(col_idx); - appender.Append(string_t(col_name)); - appender.Append(parsed_str); - - if (!options.rejects_recovery_columns.empty()) { - child_list_t recovery_key; - for (auto &key_idx : options.rejects_recovery_column_ids) { - // Figure out if the recovery key is valid. - // If not, error out for real. - auto &component_vector = parse_chunk.data[key_idx]; - if (FlatVector::IsNull(component_vector, row_idx)) { - throw InvalidInputException("%s at line %llu in column %s. Parser options:\n%s ", - "Could not parse recovery column", row_line, col_name, - options.ToString()); - } - auto component = Value(FlatVector::GetData(component_vector)[row_idx]); - recovery_key.emplace_back(names[key_idx], component); - } - appender.Append(Value::STRUCT(recovery_key)); - } - - appender.Append(string_t(row_error_msg)); - appender.EndRow(); - } - appender.Close(); - } - } - - // Now slice the insert chunk to only include the succesful rows - insert_chunk.Slice(succesful_rows, sel_size); - } - parse_chunk.Reset(); - return true; -} - -void BaseCSVReader::SetNewLineDelimiter(bool carry, bool carry_followed_by_nl) { - if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { - if (options.dialect_options.new_line == NewLineIdentifier::MIX) { - return; - } - NewLineIdentifier this_line_identifier; - if (carry) { - if (carry_followed_by_nl) { - this_line_identifier = NewLineIdentifier::CARRY_ON; - } else { - this_line_identifier = NewLineIdentifier::SINGLE; - } - } else { - this_line_identifier = NewLineIdentifier::SINGLE; - } - if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { - options.dialect_options.new_line = this_line_identifier; - return; - } - if (options.dialect_options.new_line != this_line_identifier) { - options.dialect_options.new_line = NewLineIdentifier::MIX; - return; - } - options.dialect_options.new_line = this_line_identifier; - } -} -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer.cpp similarity index 77% rename from src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp rename to src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer.cpp index 27e916e41..5c4f2e9ce 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer.cpp @@ -1,11 +1,11 @@ -#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer.hpp" #include "duckdb/common/string_util.hpp" namespace duckdb { CSVBuffer::CSVBuffer(ClientContext &context, idx_t buffer_size_p, CSVFileHandle &file_handle, idx_t &global_csv_current_position, idx_t file_number_p) - : context(context), first_buffer(true), file_number(file_number_p), can_seek(file_handle.CanSeek()) { + : context(context), file_number(file_number_p), can_seek(file_handle.CanSeek()) { AllocateBuffer(buffer_size_p); auto buffer = Ptr(); actual_buffer_size = file_handle.Read(buffer, buffer_size_p); @@ -14,17 +14,13 @@ CSVBuffer::CSVBuffer(ClientContext &context, idx_t buffer_size_p, CSVFileHandle actual_buffer_size += file_handle.Read(&buffer[actual_buffer_size], buffer_size_p - actual_buffer_size); } global_csv_start = global_csv_current_position; - // BOM check (https://en.wikipedia.org/wiki/Byte_order_mark) - if (actual_buffer_size >= 3 && buffer[0] == '\xEF' && buffer[1] == '\xBB' && buffer[2] == '\xBF') { - start_position += 3; - } last_buffer = file_handle.FinishedReading(); } CSVBuffer::CSVBuffer(CSVFileHandle &file_handle, ClientContext &context, idx_t buffer_size, - idx_t global_csv_current_position, idx_t file_number_p) + idx_t global_csv_current_position, idx_t file_number_p, idx_t buffer_idx_p) : context(context), global_csv_start(global_csv_current_position), file_number(file_number_p), - can_seek(file_handle.CanSeek()) { + can_seek(file_handle.CanSeek()), buffer_idx(buffer_idx_p) { AllocateBuffer(buffer_size); auto buffer = handle.Ptr(); actual_buffer_size = file_handle.Read(handle.Ptr(), buffer_size); @@ -36,8 +32,8 @@ CSVBuffer::CSVBuffer(CSVFileHandle &file_handle, ClientContext &context, idx_t b } shared_ptr CSVBuffer::Next(CSVFileHandle &file_handle, idx_t buffer_size, idx_t file_number_p) { - auto next_csv_buffer = - make_shared(file_handle, context, buffer_size, global_csv_start + actual_buffer_size, file_number_p); + auto next_csv_buffer = make_shared(file_handle, context, buffer_size, + global_csv_start + actual_buffer_size, file_number_p, buffer_idx + 1); if (next_csv_buffer->GetBufferSize() == 0) { // We are done reading return nullptr; @@ -68,8 +64,8 @@ unique_ptr CSVBuffer::Pin(CSVFileHandle &file_handle) { block = nullptr; Reload(file_handle); } - return make_uniq(buffer_manager.Pin(block), actual_buffer_size, first_buffer, last_buffer, - global_csv_start, start_position, file_number); + return make_uniq(buffer_manager.Pin(block), actual_buffer_size, last_buffer, file_number, + buffer_idx); } void CSVBuffer::Unpin() { @@ -78,10 +74,6 @@ void CSVBuffer::Unpin() { } } -idx_t CSVBuffer::GetStart() { - return start_position; -} - bool CSVBuffer::IsCSVFileLastBuffer() { return last_buffer; } diff --git a/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer_manager.cpp b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer_manager.cpp new file mode 100644 index 000000000..1f5b0b4db --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_buffer_manager.cpp @@ -0,0 +1,96 @@ +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer.hpp" +#include "duckdb/function/table/read_csv.hpp" +namespace duckdb { + +CSVBufferManager::CSVBufferManager(ClientContext &context_p, const CSVReaderOptions &options, const string &file_path_p, + const idx_t file_idx_p) + : context(context_p), file_idx(file_idx_p), file_path(file_path_p), buffer_size(CSVBuffer::CSV_BUFFER_SIZE) { + D_ASSERT(!file_path.empty()); + file_handle = ReadCSV::OpenCSV(file_path, options.compression, context); + skip_rows = options.dialect_options.skip_rows.GetValue(); + auto file_size = file_handle->FileSize(); + if (file_size > 0 && file_size < buffer_size) { + buffer_size = CSVBuffer::CSV_MINIMUM_BUFFER_SIZE; + } + if (options.buffer_size < buffer_size) { + buffer_size = options.buffer_size; + } + Initialize(); +} + +void CSVBufferManager::UnpinBuffer(const idx_t cache_idx) { + if (cache_idx < cached_buffers.size()) { + cached_buffers[cache_idx]->Unpin(); + } +} + +void CSVBufferManager::Initialize() { + if (cached_buffers.empty()) { + cached_buffers.emplace_back( + make_shared(context, buffer_size, *file_handle, global_csv_pos, file_idx)); + last_buffer = cached_buffers.front(); + } +} + +bool CSVBufferManager::ReadNextAndCacheIt() { + D_ASSERT(last_buffer); + for (idx_t i = 0; i < 2; i++) { + if (!last_buffer->IsCSVFileLastBuffer()) { + auto cur_buffer_size = buffer_size; + if (file_handle->uncompressed) { + if (file_handle->FileSize() - bytes_read) { + cur_buffer_size = file_handle->FileSize() - bytes_read; + } + } + if (cur_buffer_size == 0) { + last_buffer->last_buffer = true; + return false; + } + auto maybe_last_buffer = last_buffer->Next(*file_handle, cur_buffer_size, file_idx); + if (!maybe_last_buffer) { + last_buffer->last_buffer = true; + return false; + } + last_buffer = std::move(maybe_last_buffer); + bytes_read += last_buffer->GetBufferSize(); + cached_buffers.emplace_back(last_buffer); + return true; + } + } + return false; +} + +unique_ptr CSVBufferManager::GetBuffer(const idx_t pos) { + lock_guard parallel_lock(main_mutex); + while (pos >= cached_buffers.size()) { + if (done) { + return nullptr; + } + if (!ReadNextAndCacheIt()) { + done = true; + } + } + if (pos != 0) { + cached_buffers[pos - 1]->Unpin(); + } + return cached_buffers[pos]->Pin(*file_handle); +} + +idx_t CSVBufferManager::GetBufferSize() { + return buffer_size; +} + +idx_t CSVBufferManager::BufferCount() { + return cached_buffers.size(); +} + +bool CSVBufferManager::Done() { + return done; +} + +string CSVBufferManager::GetFilePath() { + return file_path; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_file_handle.cpp similarity index 93% rename from src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp rename to src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_file_handle.cpp index 6462db94d..cbb1c1cd8 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_file_handle.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/buffer_manager/csv_file_handle.cpp @@ -1,4 +1,5 @@ -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" +#include "duckdb/common/exception/binder_exception.hpp" namespace duckdb { @@ -8,6 +9,7 @@ CSVFileHandle::CSVFileHandle(FileSystem &fs, Allocator &allocator, unique_ptrCanSeek(); on_disk_file = file_handle->OnDiskFile(); file_size = file_handle->GetFileSize(); + uncompressed = compression == FileCompressionType::UNCOMPRESSED; } unique_ptr CSVFileHandle::OpenFileHandle(FileSystem &fs, Allocator &allocator, const string &path, 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 deleted file mode 100644 index 90e5403b6..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/buffered_csv_reader.cpp +++ /dev/null @@ -1,436 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" - -#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" -#include "duckdb/common/file_system.hpp" -#include "duckdb/common/string_util.hpp" -#include "duckdb/common/to_string.hpp" -#include "duckdb/common/types/cast_helpers.hpp" -#include "duckdb/common/vector_operations/unary_executor.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" -#include "duckdb/function/scalar/strftime_format.hpp" -#include "duckdb/main/client_data.hpp" -#include "duckdb/main/database.hpp" -#include "duckdb/main/error_manager.hpp" -#include "duckdb/parser/column_definition.hpp" -#include "duckdb/parser/keyword_helper.hpp" -#include "duckdb/storage/data_table.hpp" -#include "utf8proc.hpp" -#include "utf8proc_wrapper.hpp" -#include "duckdb/common/set.hpp" - -#include -#include -#include -#include - -namespace duckdb { - -BufferedCSVReader::BufferedCSVReader(ClientContext &context, CSVReaderOptions options_p, - const vector &requested_types) - : BaseCSVReader(context, std::move(options_p), requested_types), buffer_size(0), position(0), start(0) { - file_handle = OpenCSV(context, options); - Initialize(requested_types); -} - -BufferedCSVReader::BufferedCSVReader(ClientContext &context, string filename, CSVReaderOptions options_p, - const vector &requested_types) - : BaseCSVReader(context, std::move(options_p), requested_types), buffer_size(0), position(0), start(0) { - options.file_path = std::move(filename); - file_handle = OpenCSV(context, options); - Initialize(requested_types); -} - -void BufferedCSVReader::Initialize(const vector &requested_types) { - if (options.auto_detect && options.file_options.union_by_name) { - // This is required for the sniffer to work on Union By Name - D_ASSERT(options.file_path == file_handle->GetFilePath()); - auto bm_file_handle = BaseCSVReader::OpenCSV(context, options); - auto csv_buffer_manager = make_shared(context, std::move(bm_file_handle), options); - CSVSniffer sniffer(options, csv_buffer_manager, state_machine_cache); - auto sniffer_result = sniffer.SniffCSV(); - return_types = sniffer_result.return_types; - names = sniffer_result.names; - if (return_types.empty()) { - throw InvalidInputException("Failed to detect column types from CSV: is the file a valid CSV file?"); - } - } else { - return_types = requested_types; - ResetBuffer(); - } - SkipRowsAndReadHeader(options.dialect_options.skip_rows.GetValue(), options.dialect_options.header.GetValue()); - InitParseChunk(return_types.size()); -} - -void BufferedCSVReader::ResetBuffer() { - buffer.reset(); - buffer_size = 0; - position = 0; - start = 0; - cached_buffers.clear(); -} - -void BufferedCSVReader::SkipRowsAndReadHeader(idx_t skip_rows, bool skip_header) { - for (idx_t i = 0; i < skip_rows; i++) { - // ignore skip rows - string read_line = file_handle->ReadLine(); - linenr++; - } - - if (skip_header) { - // ignore the first line as a header line - InitParseChunk(return_types.size()); - ParseCSV(ParserMode::PARSING_HEADER); - } -} - -string BufferedCSVReader::ColumnTypesError(case_insensitive_map_t sql_types_per_column, - const vector &names) { - for (idx_t i = 0; i < names.size(); i++) { - auto it = sql_types_per_column.find(names[i]); - if (it != sql_types_per_column.end()) { - sql_types_per_column.erase(names[i]); - continue; - } - } - if (sql_types_per_column.empty()) { - return string(); - } - string exception = "COLUMN_TYPES error: Columns with names: "; - set problematic_columns; - for (auto &col : sql_types_per_column) { - problematic_columns.insert("\"" + col.first + "\""); - } - exception += StringUtil::Join(problematic_columns, ","); - exception += " do not exist in the CSV File"; - return exception; -} - -void BufferedCSVReader::SkipEmptyLines() { - if (parse_chunk.data.size() == 1) { - // Empty lines are null data. - return; - } - for (; position < buffer_size; position++) { - if (!StringUtil::CharacterIsNewline(buffer[position])) { - return; - } - } -} - -void UpdateMaxLineLength(ClientContext &context, idx_t line_length) { - if (!context.client_data->debug_set_max_line_length) { - return; - } - if (line_length < context.client_data->debug_max_line_length) { - return; - } - context.client_data->debug_max_line_length = line_length; -} - -bool BufferedCSVReader::ReadBuffer(idx_t &start, idx_t &line_start) { - if (start > buffer_size) { - return false; - } - auto old_buffer = std::move(buffer); - - // the remaining part of the last buffer - idx_t remaining = buffer_size - start; - - idx_t buffer_read_size = INITIAL_BUFFER_SIZE_LARGE; - - while (remaining > buffer_read_size) { - buffer_read_size *= 2; - } - - // Check line length - if (remaining > options.maximum_line_size) { - throw InvalidInputException("Maximum line size of %llu bytes exceeded on line %s!", options.maximum_line_size, - GetLineNumberStr(linenr, linenr_estimated)); - } - - buffer = make_unsafe_uniq_array(buffer_read_size + remaining + 1); - buffer_size = remaining + buffer_read_size; - if (remaining > 0) { - // remaining from last buffer: copy it here - memcpy(buffer.get(), old_buffer.get() + start, remaining); - } - idx_t read_count = file_handle->Read(buffer.get() + remaining, buffer_read_size); - - bytes_in_chunk += read_count; - buffer_size = remaining + read_count; - buffer[buffer_size] = '\0'; - if (old_buffer) { - cached_buffers.push_back(std::move(old_buffer)); - } - start = 0; - position = remaining; - if (!bom_checked) { - bom_checked = true; - if (read_count >= 3 && buffer[0] == '\xEF' && buffer[1] == '\xBB' && buffer[2] == '\xBF') { - start += 3; - position += 3; - } - } - line_start = start; - - return read_count > 0; -} - -void BufferedCSVReader::ParseCSV(DataChunk &insert_chunk) { - string error_message; - if (!TryParseCSV(ParserMode::PARSING, insert_chunk, error_message)) { - throw InvalidInputException(error_message); - } -} - -void BufferedCSVReader::ParseCSV(ParserMode mode) { - DataChunk dummy_chunk; - string error_message; - if (!TryParseCSV(mode, dummy_chunk, error_message)) { - throw InvalidInputException(error_message); - } -} - -bool BufferedCSVReader::TryParseCSV(ParserMode parser_mode, DataChunk &insert_chunk, string &error_message) { - cached_buffers.clear(); - mode = parser_mode; - // used for parsing algorithm - bool finished_chunk = false; - idx_t column = 0; - idx_t offset = 0; - bool has_quotes = false; - vector escape_positions; - - idx_t line_start = position; - idx_t line_size = 0; - // read values into the buffer (if any) - if (position >= buffer_size) { - if (!ReadBuffer(start, line_start)) { - return true; - } - } - - // start parsing the first value - goto value_start; -value_start: - offset = 0; - /* state: value_start */ - // this state parses the first character of a value - if (buffer[position] == options.dialect_options.state_machine_options.quote.GetValue()) { - // quote: actual value starts in the next position - // move to in_quotes state - start = position + 1; - line_size++; - goto in_quotes; - } else { - // no quote, move to normal parsing state - start = position; - goto normal; - } -normal: - /* state: normal parsing state */ - // this state parses the remainder of a non-quoted value until we reach a delimiter or newline - do { - for (; position < buffer_size; position++) { - line_size++; - if (buffer[position] == options.dialect_options.state_machine_options.delimiter.GetValue()) { - // delimiter: end the value and add it to the chunk - goto add_value; - } else if (StringUtil::CharacterIsNewline(buffer[position])) { - // newline: add row - goto add_row; - } - } - } while (ReadBuffer(start, line_start)); - // file ends during normal scan: go to end state - goto final_state; -add_value: - AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); - // increase position by 1 and move start to the new position - offset = 0; - has_quotes = false; - start = ++position; - line_size++; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after delimiter, go to final state - goto final_state; - } - goto value_start; -add_row : { - // check type of newline (\r or \n) - bool carriage_return = buffer[position] == '\r'; - AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); - if (!error_message.empty()) { - return false; - } - VerifyLineLength(position - line_start); - - finished_chunk = AddRow(insert_chunk, column, error_message); - UpdateMaxLineLength(context, position - line_start); - if (!error_message.empty()) { - return false; - } - // increase position by 1 and move start to the new position - offset = 0; - has_quotes = false; - position++; - line_size = 0; - start = position; - line_start = position; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after delimiter, go to final state - goto final_state; - } - if (carriage_return) { - // \r newline, go to special state that parses an optional \n afterwards - goto carriage_return; - } else { - SetNewLineDelimiter(); - SkipEmptyLines(); - - start = position; - line_start = position; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after delimiter, go to final state - goto final_state; - } - // \n newline, move to value start - if (finished_chunk) { - return true; - } - goto value_start; - } -} -in_quotes: - /* state: in_quotes */ - // this state parses the remainder of a quoted value - has_quotes = true; - position++; - line_size++; - do { - for (; position < buffer_size; position++) { - line_size++; - if (buffer[position] == options.dialect_options.state_machine_options.quote.GetValue()) { - // quote: move to unquoted state - goto unquote; - } else if (buffer[position] == options.dialect_options.state_machine_options.escape.GetValue()) { - // escape: store the escaped position and move to handle_escape state - escape_positions.push_back(position - start); - goto handle_escape; - } - } - } while (ReadBuffer(start, line_start)); - // still in quoted state at the end of the file, error: - throw InvalidInputException("Error in file \"%s\" on line %s: unterminated quotes. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); -unquote: - /* state: unquote */ - // this state handles the state directly after we unquote - // in this state we expect either another quote (entering the quoted state again, and escaping the quote) - // or a delimiter/newline, ending the current value and moving on to the next value - position++; - line_size++; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after unquote, go to final state - offset = 1; - goto final_state; - } - if (buffer[position] == options.dialect_options.state_machine_options.quote.GetValue() && - (options.dialect_options.state_machine_options.escape == '\0' || - options.dialect_options.state_machine_options.escape == options.dialect_options.state_machine_options.quote)) { - // escaped quote, return to quoted state and store escape position - escape_positions.push_back(position - start); - goto in_quotes; - } else if (options.dialect_options.state_machine_options.delimiter == buffer[position]) { - // delimiter, add value - offset = 1; - goto add_value; - } else if (StringUtil::CharacterIsNewline(buffer[position])) { - offset = 1; - goto add_row; - } else { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: quote should be followed by end of value, end of " - "row or another quote. (%s)", - options.file_path, GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); - return false; - } -handle_escape: - /* state: handle_escape */ - // escape should be followed by a quote or another escape character - position++; - line_size++; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); - return false; - } - if (options.dialect_options.state_machine_options.quote != buffer[position] && - options.dialect_options.state_machine_options.escape != buffer[position]) { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated).c_str(), options.ToString()); - return false; - } - // escape was followed by quote or escape, go back to quoted state - goto in_quotes; -carriage_return: - /* state: carriage_return */ - // this stage optionally skips a newline (\n) character, which allows \r\n to be interpreted as a single line - if (buffer[position] == '\n') { - SetNewLineDelimiter(true, true); - // newline after carriage return: skip - // increase position by 1 and move start to the new position - start = ++position; - line_size++; - - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after delimiter, go to final state - goto final_state; - } - } else { - SetNewLineDelimiter(true, false); - } - if (finished_chunk) { - return true; - } - SkipEmptyLines(); - start = position; - line_start = position; - if (position >= buffer_size && !ReadBuffer(start, line_start)) { - // file ends right after delimiter, go to final state - goto final_state; - } - - goto value_start; -final_state: - if (finished_chunk) { - return true; - } - - if (column > 0 || position > start) { - // remaining values to be added to the chunk - AddValue(string_t(buffer.get() + start, position - start - offset), column, escape_positions, has_quotes); - VerifyLineLength(position - line_start); - - finished_chunk = AddRow(insert_chunk, column, error_message); - SkipEmptyLines(); - UpdateMaxLineLength(context, line_size); - if (!error_message.empty()) { - return false; - } - } - - // final stage, only reached after parsing the file is finished - // flush the parsed chunk and finalize parsing - if (mode == ParserMode::PARSING) { - Flush(insert_chunk); - } - - return true; -} - -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp deleted file mode 100644 index a63c6d714..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_buffer_manager.cpp +++ /dev/null @@ -1,89 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" -namespace duckdb { - -CSVBufferManager::CSVBufferManager(ClientContext &context_p, unique_ptr file_handle_p, - const CSVReaderOptions &options, idx_t file_idx_p) - : file_handle(std::move(file_handle_p)), context(context_p), file_idx(file_idx_p), - buffer_size(CSVBuffer::CSV_BUFFER_SIZE) { - // Skip rows if they are set - skip_rows = options.dialect_options.skip_rows.GetValue(); - - auto file_size = file_handle->FileSize(); - if (file_size > 0 && file_size < buffer_size) { - buffer_size = CSVBuffer::CSV_MINIMUM_BUFFER_SIZE; - } - if (options.buffer_size < buffer_size) { - buffer_size = options.buffer_size; - } - for (idx_t i = 0; i < skip_rows; i++) { - file_handle->ReadLine(); - } - Initialize(); -} - -void CSVBufferManager::UnpinBuffer(idx_t cache_idx) { - if (cache_idx < cached_buffers.size()) { - cached_buffers[cache_idx]->Unpin(); - } -} - -void CSVBufferManager::Initialize() { - if (cached_buffers.empty()) { - cached_buffers.emplace_back( - make_shared(context, buffer_size, *file_handle, global_csv_pos, file_idx)); - last_buffer = cached_buffers.front(); - } - start_pos = last_buffer->GetStart(); -} - -idx_t CSVBufferManager::GetStartPos() { - return start_pos; -} -bool CSVBufferManager::ReadNextAndCacheIt() { - D_ASSERT(last_buffer); - if (!last_buffer->IsCSVFileLastBuffer()) { - auto maybe_last_buffer = last_buffer->Next(*file_handle, buffer_size, file_idx); - if (!maybe_last_buffer) { - last_buffer->last_buffer = true; - return false; - } - last_buffer = std::move(maybe_last_buffer); - cached_buffers.emplace_back(last_buffer); - return true; - } - return false; -} - -unique_ptr CSVBufferManager::GetBuffer(const idx_t pos) { - while (pos >= cached_buffers.size()) { - if (done) { - return nullptr; - } - if (!ReadNextAndCacheIt()) { - done = true; - } - } - if (pos != 0) { - cached_buffers[pos - 1]->Unpin(); - } - return cached_buffers[pos]->Pin(*file_handle); -} - -bool CSVBufferIterator::Finished() { - return !cur_buffer_handle; -} - -void CSVBufferIterator::Reset() { - if (cur_buffer_handle) { - cur_buffer_handle.reset(); - } - if (cur_buffer_idx > 0) { - buffer_manager->UnpinBuffer(cur_buffer_idx - 1); - } - cur_buffer_idx = 0; - buffer_manager->Initialize(); - cur_pos = buffer_manager->GetStartPos(); -} - -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp deleted file mode 100644 index f7b56527b..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine.cpp +++ /dev/null @@ -1,34 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" -#include "utf8proc_wrapper.hpp" -#include "duckdb/main/error_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" - -namespace duckdb { - -CSVStateMachine::CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options, - shared_ptr buffer_manager_p, - CSVStateMachineCache &csv_state_machine_cache_p) - : csv_state_machine_cache(csv_state_machine_cache_p), options(options_p), - csv_buffer_iterator(std::move(buffer_manager_p)), - transition_array(csv_state_machine_cache.Get(state_machine_options)) { - dialect_options.state_machine_options = state_machine_options; - dialect_options.date_format = options.dialect_options.date_format; - dialect_options.skip_rows = options.dialect_options.skip_rows; -} - -void CSVStateMachine::Reset() { - csv_buffer_iterator.Reset(); -} - -void CSVStateMachine::VerifyUTF8() { - auto utf_type = Utf8Proc::Analyze(value.c_str(), value.size()); - if (utf_type == UnicodeType::INVALID) { - int64_t error_line = cur_rows; - throw InvalidInputException("Error in file \"%s\" at line %llu: " - "%s. Parser options:\n%s", - options.file_path, error_line, ErrorManager::InvalidUnicodeError(value, "CSV file"), - options.ToString()); - } -} -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp b/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp deleted file mode 100644 index b2e6767ec..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_state_machine_cache.cpp +++ /dev/null @@ -1,102 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" - -namespace duckdb { - -void InitializeTransitionArray(CSVState *transition_array, const CSVState state) { - for (uint32_t i = 0; i < StateMachine::NUM_TRANSITIONS; i++) { - transition_array[i] = state; - } -} - -void CSVStateMachineCache::Insert(const CSVStateMachineOptions &state_machine_options) { - D_ASSERT(state_machine_cache.find(state_machine_options) == state_machine_cache.end()); - // Initialize transition array with default values to the Standard option - auto &transition_array = state_machine_cache[state_machine_options]; - - for (uint32_t i = 0; i < StateMachine::NUM_STATES; i++) { - CSVState cur_state = CSVState(i); - switch (cur_state) { - case CSVState::QUOTED: - InitializeTransitionArray(transition_array[cur_state], CSVState::QUOTED); - break; - case CSVState::UNQUOTED: - case CSVState::INVALID: - case CSVState::ESCAPE: - InitializeTransitionArray(transition_array[cur_state], CSVState::INVALID); - break; - default: - InitializeTransitionArray(transition_array[cur_state], CSVState::STANDARD); - break; - } - } - - uint8_t delimiter = static_cast(state_machine_options.delimiter.GetValue()); - uint8_t quote = static_cast(state_machine_options.quote.GetValue()); - uint8_t escape = static_cast(state_machine_options.escape.GetValue()); - - // Now set values depending on configuration - // 1) Standard State - transition_array[CSVState::STANDARD][delimiter] = CSVState::DELIMITER; - transition_array[CSVState::STANDARD][static_cast('\n')] = CSVState::RECORD_SEPARATOR; - transition_array[CSVState::STANDARD][static_cast('\r')] = CSVState::CARRIAGE_RETURN; - transition_array[CSVState::STANDARD][quote] = CSVState::QUOTED; - // 2) Field Separator State - transition_array[CSVState::DELIMITER][delimiter] = CSVState::DELIMITER; - transition_array[CSVState::DELIMITER][static_cast('\n')] = CSVState::RECORD_SEPARATOR; - transition_array[CSVState::DELIMITER][static_cast('\r')] = CSVState::CARRIAGE_RETURN; - transition_array[CSVState::DELIMITER][quote] = CSVState::QUOTED; - // 3) Record Separator State - transition_array[CSVState::RECORD_SEPARATOR][delimiter] = CSVState::DELIMITER; - transition_array[CSVState::RECORD_SEPARATOR][static_cast('\n')] = CSVState::EMPTY_LINE; - transition_array[CSVState::RECORD_SEPARATOR][static_cast('\r')] = CSVState::EMPTY_LINE; - transition_array[CSVState::RECORD_SEPARATOR][quote] = CSVState::QUOTED; - // 4) Carriage Return State - transition_array[CSVState::CARRIAGE_RETURN][static_cast('\n')] = CSVState::RECORD_SEPARATOR; - transition_array[CSVState::CARRIAGE_RETURN][static_cast('\r')] = CSVState::EMPTY_LINE; - transition_array[CSVState::CARRIAGE_RETURN][escape] = CSVState::ESCAPE; - // 5) Quoted State - transition_array[CSVState::QUOTED][quote] = CSVState::UNQUOTED; - if (state_machine_options.quote != state_machine_options.escape) { - transition_array[CSVState::QUOTED][escape] = CSVState::ESCAPE; - } - // 6) Unquoted State - transition_array[CSVState::UNQUOTED][static_cast('\n')] = CSVState::RECORD_SEPARATOR; - transition_array[CSVState::UNQUOTED][static_cast('\r')] = CSVState::CARRIAGE_RETURN; - transition_array[CSVState::UNQUOTED][delimiter] = CSVState::DELIMITER; - if (state_machine_options.quote == state_machine_options.escape) { - transition_array[CSVState::UNQUOTED][escape] = CSVState::QUOTED; - } - // 7) Escaped State - transition_array[CSVState::ESCAPE][quote] = CSVState::QUOTED; - transition_array[CSVState::ESCAPE][escape] = CSVState::QUOTED; - // 8) Empty Line State - transition_array[CSVState::EMPTY_LINE][static_cast('\r')] = CSVState::EMPTY_LINE; - transition_array[CSVState::EMPTY_LINE][static_cast('\n')] = CSVState::EMPTY_LINE; - transition_array[CSVState::EMPTY_LINE][delimiter] = CSVState::DELIMITER; - transition_array[CSVState::EMPTY_LINE][quote] = CSVState::QUOTED; -} - -CSVStateMachineCache::CSVStateMachineCache() { - for (auto quoterule : default_quote_rule) { - const auto "e_candidates = default_quote[static_cast(quoterule)]; - for (const auto "e : quote_candidates) { - for (const auto &delimiter : default_delimiter) { - const auto &escape_candidates = default_escape[static_cast(quoterule)]; - for (const auto &escape : escape_candidates) { - Insert({delimiter, quote, escape}); - } - } - } - } -} - -const StateMachine &CSVStateMachineCache::Get(const CSVStateMachineOptions &state_machine_options) { - //! Custom State Machine, we need to create it and cache it first - if (state_machine_cache.find(state_machine_options) == state_machine_cache.end()) { - Insert(state_machine_options); - } - const auto &transition_array = state_machine_cache[state_machine_options]; - return transition_array; -} -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp b/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp deleted file mode 100644 index 33eae47a0..000000000 --- a/src/duckdb/src/execution/operator/csv_scanner/parallel_csv_reader.cpp +++ /dev/null @@ -1,689 +0,0 @@ -#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" - -#include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" -#include "duckdb/common/file_system.hpp" -#include "duckdb/common/string_util.hpp" -#include "duckdb/common/to_string.hpp" -#include "duckdb/common/types/cast_helpers.hpp" -#include "duckdb/common/vector_operations/unary_executor.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" -#include "duckdb/function/scalar/strftime_format.hpp" -#include "duckdb/main/database.hpp" -#include "duckdb/parser/column_definition.hpp" -#include "duckdb/storage/data_table.hpp" -#include "utf8proc_wrapper.hpp" -#include "utf8proc.hpp" -#include "duckdb/parser/keyword_helper.hpp" -#include "duckdb/function/table/read_csv.hpp" -#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" - -#include -#include -#include -#include - -namespace duckdb { - -ParallelCSVReader::ParallelCSVReader(ClientContext &context, CSVReaderOptions options_p, - unique_ptr buffer_p, idx_t first_pos_first_buffer_p, - const vector &requested_types, idx_t file_idx_p) - : BaseCSVReader(context, std::move(options_p), requested_types), file_idx(file_idx_p), - first_pos_first_buffer(first_pos_first_buffer_p) { - Initialize(requested_types); - SetBufferRead(std::move(buffer_p)); -} - -void ParallelCSVReader::Initialize(const vector &requested_types) { - return_types = requested_types; - InitParseChunk(return_types.size()); -} - -bool ParallelCSVReader::NewLineDelimiter(bool carry, bool carry_followed_by_nl, bool first_char) { - // Set the delimiter if not set yet. - SetNewLineDelimiter(carry, carry_followed_by_nl); - D_ASSERT(options.dialect_options.new_line == NewLineIdentifier::SINGLE || - options.dialect_options.new_line == NewLineIdentifier::CARRY_ON); - if (options.dialect_options.new_line == NewLineIdentifier::SINGLE) { - return (!carry) || (carry && !carry_followed_by_nl); - } - return (carry && carry_followed_by_nl) || (!carry && first_char); -} - -bool ParallelCSVReader::SkipEmptyLines() { - const idx_t initial_position_buffer = position_buffer; - idx_t new_pos_buffer = position_buffer; - if (parse_chunk.data.size() == 1) { - // Empty lines are null data. - return initial_position_buffer != position_buffer; - } - for (; new_pos_buffer < end_buffer; new_pos_buffer++) { - if (StringUtil::CharacterIsNewline((*buffer)[new_pos_buffer])) { - bool carrier_return = (*buffer)[new_pos_buffer] == '\r'; - new_pos_buffer++; - if (carrier_return && new_pos_buffer < buffer_size && (*buffer)[new_pos_buffer] == '\n') { - position_buffer++; - } - if (new_pos_buffer > end_buffer) { - return initial_position_buffer != position_buffer; - } - position_buffer = new_pos_buffer; - } else if ((*buffer)[new_pos_buffer] != ' ') { - return initial_position_buffer != position_buffer; - } - } - return initial_position_buffer != position_buffer; -} - -bool ParallelCSVReader::SetPosition() { - if (buffer->buffer->is_first_buffer && start_buffer == position_buffer && start_buffer == first_pos_first_buffer) { - start_buffer = buffer->buffer->start_position; - position_buffer = start_buffer; - verification_positions.beginning_of_first_line = position_buffer; - verification_positions.end_of_last_line = position_buffer; - // First buffer doesn't need any setting - - if (options.dialect_options.header.GetValue()) { - for (; position_buffer < end_buffer; position_buffer++) { - if (StringUtil::CharacterIsNewline((*buffer)[position_buffer])) { - bool carrier_return = (*buffer)[position_buffer] == '\r'; - position_buffer++; - if (carrier_return && position_buffer < buffer_size && (*buffer)[position_buffer] == '\n') { - position_buffer++; - } - if (position_buffer > end_buffer) { - VerifyLineLength(position_buffer, buffer->batch_index); - return false; - } - SkipEmptyLines(); - if (verification_positions.beginning_of_first_line == 0) { - verification_positions.beginning_of_first_line = position_buffer; - } - VerifyLineLength(position_buffer, buffer->batch_index); - verification_positions.end_of_last_line = position_buffer; - return true; - } - } - VerifyLineLength(position_buffer, buffer->batch_index); - return false; - } - SkipEmptyLines(); - if (verification_positions.beginning_of_first_line == 0) { - verification_positions.beginning_of_first_line = position_buffer; - } - - verification_positions.end_of_last_line = position_buffer; - return true; - } - - // We have to move position up to next new line - idx_t end_buffer_real = end_buffer; - // Check if we already start in a valid line - string error_message; - bool successfully_read_first_line = false; - while (!successfully_read_first_line) { - DataChunk first_line_chunk; - first_line_chunk.Initialize(allocator, return_types); - // Ensure that parse_chunk has no gunk when trying to figure new line - parse_chunk.Reset(); - for (; position_buffer < end_buffer; position_buffer++) { - if (StringUtil::CharacterIsNewline((*buffer)[position_buffer])) { - bool carriage_return = (*buffer)[position_buffer] == '\r'; - bool carriage_return_followed = false; - position_buffer++; - if (position_buffer < end_buffer) { - if (carriage_return && (*buffer)[position_buffer] == '\n') { - carriage_return_followed = true; - position_buffer++; - } - } - if (NewLineDelimiter(carriage_return, carriage_return_followed, position_buffer - 1 == start_buffer)) { - break; - } - } - } - SkipEmptyLines(); - - if (position_buffer > buffer_size) { - break; - } - - auto pos_check = position_buffer == 0 ? position_buffer : position_buffer - 1; - if (position_buffer >= end_buffer && !StringUtil::CharacterIsNewline((*buffer)[pos_check])) { - break; - } - - if (position_buffer > end_buffer && options.dialect_options.new_line == NewLineIdentifier::CARRY_ON && - (*buffer)[pos_check] == '\n') { - break; - } - idx_t position_set = position_buffer; - start_buffer = position_buffer; - // We check if we can add this line - // disable the projection pushdown while reading the first line - // otherwise the first line parsing can be influenced by which columns we are reading - auto column_ids = std::move(reader_data.column_ids); - auto column_mapping = std::move(reader_data.column_mapping); - InitializeProjection(); - try { - successfully_read_first_line = TryParseSimpleCSV(first_line_chunk, error_message, true); - } catch (...) { - successfully_read_first_line = false; - } - // restore the projection pushdown - reader_data.column_ids = std::move(column_ids); - reader_data.column_mapping = std::move(column_mapping); - end_buffer = end_buffer_real; - start_buffer = position_set; - if (position_buffer >= end_buffer) { - if (successfully_read_first_line) { - position_buffer = position_set; - } - break; - } - position_buffer = position_set; - } - if (verification_positions.beginning_of_first_line == 0) { - verification_positions.beginning_of_first_line = position_buffer; - } - // Ensure that parse_chunk has no gunk when trying to figure new line - parse_chunk.Reset(); - verification_positions.end_of_last_line = position_buffer; - finished = false; - return successfully_read_first_line; -} - -void ParallelCSVReader::SetBufferRead(unique_ptr buffer_read_p) { - if (!buffer_read_p->buffer) { - throw InternalException("ParallelCSVReader::SetBufferRead - CSVBufferRead does not have a buffer to read"); - } - position_buffer = buffer_read_p->buffer_start; - start_buffer = buffer_read_p->buffer_start; - end_buffer = buffer_read_p->buffer_end; - if (buffer_read_p->next_buffer) { - buffer_size = buffer_read_p->buffer->actual_size + buffer_read_p->next_buffer->actual_size; - } else { - buffer_size = buffer_read_p->buffer->actual_size; - } - buffer = std::move(buffer_read_p); - - reached_remainder_state = false; - verification_positions.beginning_of_first_line = 0; - verification_positions.end_of_last_line = 0; - finished = false; - D_ASSERT(end_buffer <= buffer_size); -} - -VerificationPositions ParallelCSVReader::GetVerificationPositions() { - verification_positions.beginning_of_first_line += buffer->buffer->csv_global_start; - verification_positions.end_of_last_line += buffer->buffer->csv_global_start; - return verification_positions; -} - -// If BufferRemainder returns false, it means we are done scanning this buffer and should go to the end_state -bool ParallelCSVReader::BufferRemainder() { - if (position_buffer >= end_buffer && !reached_remainder_state) { - // First time we finish the buffer piece we should scan here, we set the variables - // to allow this piece to be scanned up to the end of the buffer or the next new line - reached_remainder_state = true; - // end_buffer is allowed to go to buffer size to finish its last line - end_buffer = buffer_size; - } - if (position_buffer >= end_buffer) { - // buffer ends, return false - return false; - } - // we can still scan stuff, return true - return true; -} - -bool AllNewLine(string_t value, idx_t column_amount) { - auto value_str = value.GetString(); - if (value_str.empty() && column_amount == 1) { - // This is a one column (empty) - return false; - } - for (idx_t i = 0; i < value.GetSize(); i++) { - if (!StringUtil::CharacterIsNewline(value_str[i])) { - return false; - } - } - return true; -} - -bool ParallelCSVReader::TryParseSimpleCSV(DataChunk &insert_chunk, string &error_message, bool try_add_line) { - // If line is not set, we have to figure it out, we assume whatever is in the first line - if (options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { - idx_t cur_pos = position_buffer; - // we can start in the middle of a new line, so move a bit forward. - while (cur_pos < end_buffer) { - if (StringUtil::CharacterIsNewline((*buffer)[cur_pos])) { - cur_pos++; - } else { - break; - } - } - for (; cur_pos < end_buffer; cur_pos++) { - if (StringUtil::CharacterIsNewline((*buffer)[cur_pos])) { - bool carriage_return = (*buffer)[cur_pos] == '\r'; - bool carriage_return_followed = false; - cur_pos++; - if (cur_pos < end_buffer) { - if (carriage_return && (*buffer)[cur_pos] == '\n') { - carriage_return_followed = true; - cur_pos++; - } - } - SetNewLineDelimiter(carriage_return, carriage_return_followed); - break; - } - } - } - // used for parsing algorithm - if (start_buffer == buffer_size) { - // Nothing to read - finished = true; - return true; - } - D_ASSERT(end_buffer <= buffer_size); - bool finished_chunk = false; - idx_t column = 0; - idx_t offset = 0; - bool has_quotes = false; - bool last_line_empty = false; - vector escape_positions; - if ((start_buffer == buffer->buffer_start || start_buffer == buffer->buffer_end) && !try_add_line) { - // First time reading this buffer piece - if (!SetPosition()) { - finished = true; - return true; - } - } - if (position_buffer == buffer_size) { - // Nothing to read - finished = true; - return true; - } - // Keep track of line size - idx_t line_start = position_buffer; - // start parsing the first value - goto value_start; - -value_start : { - /* state: value_start */ - if (!BufferRemainder()) { - goto final_state; - } - offset = 0; - - // this state parses the first character of a value - if (options.dialect_options.state_machine_options.quote == (*buffer)[position_buffer]) { - // quote: actual value starts in the next position - // move to in_quotes state - start_buffer = position_buffer + 1; - goto in_quotes; - } else { - // no quote, move to normal parsing state - start_buffer = position_buffer; - goto normal; - } -}; - -normal : { - /* state: normal parsing state */ - // this state parses the remainder of a non-quoted value until we reach a delimiter or newline - for (; position_buffer < end_buffer; position_buffer++) { - auto c = (*buffer)[position_buffer]; - if (options.dialect_options.state_machine_options.delimiter == c) { - // Check if previous character is a quote, if yes, this means we are in a non-initialized quoted value - // This only matters for when trying to figure out where csv lines start - if (position_buffer > 0 && try_add_line) { - if (options.dialect_options.state_machine_options.quote == (*buffer)[position_buffer - 1]) { - return false; - } - } - // delimiter: end the value and add it to the chunk - goto add_value; - } else if (StringUtil::CharacterIsNewline(c)) { - // Check if previous character is a quote, if yes, this means we are in a non-initialized quoted value - // This only matters for when trying to figure out where csv lines start - if (position_buffer > 0 && try_add_line) { - if (options.dialect_options.state_machine_options.quote == (*buffer)[position_buffer - 1]) { - return false; - } - } - // newline: add row - if (column > 0 || try_add_line || parse_chunk.data.size() == 1) { - goto add_row; - } - if (column == 0 && position_buffer == start_buffer) { - start_buffer++; - } - } - } - if (!BufferRemainder()) { - goto final_state; - } else { - goto normal; - } -}; - -add_value : { - /* state: Add value to string vector */ - AddValue(buffer->GetValue(start_buffer, position_buffer, offset), column, escape_positions, has_quotes, - buffer->local_batch_index); - // increase position by 1 and move start to the new position - offset = 0; - has_quotes = false; - start_buffer = ++position_buffer; - if (!BufferRemainder()) { - goto final_state; - } - goto value_start; -}; - -add_row : { - /* state: Add Row to Parse chunk */ - // check type of newline (\r or \n) - bool carriage_return = (*buffer)[position_buffer] == '\r'; - - AddValue(buffer->GetValue(start_buffer, position_buffer, offset), column, escape_positions, has_quotes, - buffer->local_batch_index); - if (try_add_line) { - bool success = column == insert_chunk.ColumnCount(); - if (success) { - idx_t cur_linenr = linenr; - AddRow(insert_chunk, column, error_message, buffer->local_batch_index); - success = Flush(insert_chunk, buffer->local_batch_index, true); - linenr = cur_linenr; - } - reached_remainder_state = false; - parse_chunk.Reset(); - return success; - } else { - VerifyLineLength(position_buffer - line_start, buffer->batch_index); - line_start = position_buffer; - finished_chunk = AddRow(insert_chunk, column, error_message, buffer->local_batch_index); - } - // increase position by 1 and move start to the new position - offset = 0; - has_quotes = false; - position_buffer++; - start_buffer = position_buffer; - verification_positions.end_of_last_line = position_buffer; - if (carriage_return) { - // \r newline, go to special state that parses an optional \n afterwards - // optionally skips a newline (\n) character, which allows \r\n to be interpreted as a single line - if (!BufferRemainder()) { - goto final_state; - } - if ((*buffer)[position_buffer] == '\n') { - if (options.dialect_options.new_line == NewLineIdentifier::SINGLE) { - error_message = "Wrong NewLine Identifier. Expecting \\r\\n"; - return false; - } - // newline after carriage return: skip - // increase position by 1 and move start to the new position - start_buffer = ++position_buffer; - - SkipEmptyLines(); - verification_positions.end_of_last_line = position_buffer; - start_buffer = position_buffer; - if (reached_remainder_state) { - goto final_state; - } - } else { - if (options.dialect_options.new_line == NewLineIdentifier::CARRY_ON) { - error_message = "Wrong NewLine Identifier. Expecting \\r or \\n"; - return false; - } - } - if (!BufferRemainder()) { - goto final_state; - } - if (reached_remainder_state || finished_chunk) { - goto final_state; - } - goto value_start; - } else { - if (options.dialect_options.new_line == NewLineIdentifier::CARRY_ON) { - error_message = "Wrong NewLine Identifier. Expecting \\r or \\n"; - return false; - } - if (reached_remainder_state) { - goto final_state; - } - if (!BufferRemainder()) { - goto final_state; - } - if (SkipEmptyLines() && reached_remainder_state) { - last_line_empty = true; - goto final_state; - } - if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { - error_message = "Line does not fit in one buffer. Increase the buffer size."; - return false; - } - verification_positions.end_of_last_line = position_buffer; - start_buffer = position_buffer; - // \n newline, move to value start - if (finished_chunk) { - goto final_state; - } - goto value_start; - } -} -in_quotes: - /* state: in_quotes this state parses the remainder of a quoted value*/ - has_quotes = true; - position_buffer++; - for (; position_buffer < end_buffer; position_buffer++) { - auto c = (*buffer)[position_buffer]; - if (options.dialect_options.state_machine_options.quote == c) { - // quote: move to unquoted state - goto unquote; - } else if (options.dialect_options.state_machine_options.escape == c) { - // escape: store the escaped position and move to handle_escape state - escape_positions.push_back(position_buffer - start_buffer); - goto handle_escape; - } - } - if (!BufferRemainder()) { - if (buffer->buffer->is_last_buffer) { - if (try_add_line) { - return false; - } - // still in quoted state at the end of the file or at the end of a buffer when running multithreaded, error: - throw InvalidInputException("Error in file \"%s\" on line %s: unterminated quotes. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), - options.ToString()); - } else { - goto final_state; - } - } else { - position_buffer--; - goto in_quotes; - } - -unquote : { - /* state: unquote: this state handles the state directly after we unquote*/ - // - // in this state we expect either another quote (entering the quoted state again, and escaping the quote) - // or a delimiter/newline, ending the current value and moving on to the next value - position_buffer++; - if (!BufferRemainder()) { - offset = 1; - goto final_state; - } - auto c = (*buffer)[position_buffer]; - if (options.dialect_options.state_machine_options.quote == c && - (options.dialect_options.state_machine_options.escape == '\0' || - options.dialect_options.state_machine_options.escape == options.dialect_options.state_machine_options.quote)) { - // escaped quote, return to quoted state and store escape position - escape_positions.push_back(position_buffer - start_buffer); - goto in_quotes; - } else if (options.dialect_options.state_machine_options.delimiter == c) { - // delimiter, add value - offset = 1; - goto add_value; - } else if (StringUtil::CharacterIsNewline(c)) { - offset = 1; - // FIXME: should this be an assertion? - D_ASSERT(try_add_line || (!try_add_line && column == parse_chunk.ColumnCount() - 1)); - goto add_row; - } else if (position_buffer >= end_buffer) { - // reached end of buffer - offset = 1; - goto final_state; - } else { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: quote should be followed by end of value, end of " - "row or another quote. (%s). ", - options.file_path, GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), - options.ToString()); - return false; - } -} -handle_escape : { - /* state: handle_escape */ - // escape should be followed by a quote or another escape character - position_buffer++; - if (!BufferRemainder()) { - goto final_state; - } - if (position_buffer >= buffer_size && buffer->buffer->is_last_buffer) { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), options.ToString()); - return false; - } - if (options.dialect_options.state_machine_options.quote != (*buffer)[position_buffer] && - options.dialect_options.state_machine_options.escape != (*buffer)[position_buffer]) { - error_message = StringUtil::Format( - "Error in file \"%s\" on line %s: neither QUOTE nor ESCAPE is proceeded by ESCAPE. (%s)", options.file_path, - GetLineNumberStr(linenr, linenr_estimated, buffer->local_batch_index).c_str(), options.ToString()); - return false; - } - // escape was followed by quote or escape, go back to quoted state - goto in_quotes; -} -final_state : { - /* state: final_stage reached after we finished reading the end_buffer of the csv buffer */ - // reset end buffer - end_buffer = buffer->buffer_end; - if (position_buffer == end_buffer) { - reached_remainder_state = false; - } - if (finished_chunk) { - if (position_buffer >= end_buffer) { - if (position_buffer == end_buffer && StringUtil::CharacterIsNewline((*buffer)[position_buffer - 1]) && - position_buffer < buffer_size) { - // last position is a new line, we still have to go through one more line of this buffer - finished = false; - } else { - finished = true; - } - } - buffer->lines_read += insert_chunk.size(); - return true; - } - // If this is the last buffer, we have to read the last value - if (!last_line_empty && (buffer->buffer->is_last_buffer || !buffer->next_buffer || - (buffer->next_buffer && buffer->next_buffer->is_last_buffer))) { - if (column > 0 || start_buffer != position_buffer || try_add_line || - (insert_chunk.data.size() == 1 && start_buffer != position_buffer)) { - // remaining values to be added to the chunk - auto str_value = buffer->GetValue(start_buffer, position_buffer, offset); - if (!AllNewLine(str_value, insert_chunk.data.size()) || offset == 0) { - AddValue(str_value, column, escape_positions, has_quotes, buffer->local_batch_index); - if (try_add_line) { - bool success = column == return_types.size(); - if (success) { - auto cur_linenr = linenr; - AddRow(insert_chunk, column, error_message, buffer->local_batch_index); - success = Flush(insert_chunk, buffer->local_batch_index); - linenr = cur_linenr; - } - parse_chunk.Reset(); - reached_remainder_state = false; - return success; - } else { - VerifyLineLength(position_buffer - line_start, buffer->batch_index); - line_start = position_buffer; - AddRow(insert_chunk, column, error_message, buffer->local_batch_index); - if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { - error_message = "Line does not fit in one buffer. Increase the buffer size."; - return false; - } - verification_positions.end_of_last_line = position_buffer; - } - } - } - } - // flush the parsed chunk and finalize parsing - if (mode == ParserMode::PARSING) { - Flush(insert_chunk, buffer->local_batch_index); - buffer->lines_read += insert_chunk.size(); - } - if (position_buffer - verification_positions.end_of_last_line > options.buffer_size) { - error_message = "Line does not fit in one buffer. Increase the buffer size."; - return false; - } - end_buffer = buffer_size; - SkipEmptyLines(); - end_buffer = buffer->buffer_end; - verification_positions.end_of_last_line = position_buffer; - if (position_buffer >= end_buffer) { - if (position_buffer >= end_buffer) { - if (position_buffer == end_buffer && StringUtil::CharacterIsNewline((*buffer)[position_buffer - 1]) && - position_buffer < buffer_size) { - // last position is a new line, we still have to go through one more line of this buffer - finished = false; - } else { - finished = true; - } - } - } - return true; -}; -} - -void ParallelCSVReader::ParseCSV(DataChunk &insert_chunk) { - string error_message; - if (!TryParseCSV(ParserMode::PARSING, insert_chunk, error_message)) { - throw InvalidInputException(error_message); - } -} - -idx_t ParallelCSVReader::GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first) { - while (true) { - if (buffer->line_info->CanItGetLine(file_idx, buffer_idx)) { - auto cur_start = verification_positions.beginning_of_first_line + buffer->buffer->csv_global_start; - return buffer->line_info->GetLine(buffer_idx, line_error, file_idx, cur_start, false, stop_at_first); - } - } -} - -void ParallelCSVReader::Increment(idx_t buffer_idx) { - return buffer->line_info->Increment(file_idx, buffer_idx); -} - -bool ParallelCSVReader::TryParseCSV(ParserMode mode) { - DataChunk dummy_chunk; - string error_message; - return TryParseCSV(mode, dummy_chunk, error_message); -} - -void ParallelCSVReader::ParseCSV(ParserMode mode) { - DataChunk dummy_chunk; - string error_message; - if (!TryParseCSV(mode, dummy_chunk, error_message)) { - throw InvalidInputException(error_message); - } -} - -bool ParallelCSVReader::TryParseCSV(ParserMode parser_mode, DataChunk &insert_chunk, string &error_message) { - mode = parser_mode; - return TryParseSimpleCSV(insert_chunk, error_message); -} - -} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/scanner/base_scanner.cpp b/src/duckdb/src/execution/operator/csv_scanner/scanner/base_scanner.cpp new file mode 100644 index 000000000..33746dba6 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/scanner/base_scanner.cpp @@ -0,0 +1,71 @@ +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/base_scanner.hpp" + +namespace duckdb { + +ScannerResult::ScannerResult(CSVStates &states_p, CSVStateMachine &state_machine_p) + : states(states_p), state_machine(state_machine_p) { +} + +BaseScanner::BaseScanner(shared_ptr buffer_manager_p, shared_ptr state_machine_p, + shared_ptr error_handler_p, shared_ptr csv_file_scan_p, + CSVIterator iterator_p) + : csv_file_scan(std::move(csv_file_scan_p)), error_handler(std::move(error_handler_p)), + state_machine(std::move(state_machine_p)), iterator(iterator_p), buffer_manager(std::move(buffer_manager_p)) { + D_ASSERT(buffer_manager); + D_ASSERT(state_machine); + // Initialize current buffer handle + cur_buffer_handle = buffer_manager->GetBuffer(iterator.GetBufferIdx()); + if (!cur_buffer_handle) { + buffer_handle_ptr = nullptr; + } else { + buffer_handle_ptr = cur_buffer_handle->Ptr(); + } +} + +bool BaseScanner::FinishedFile() { + if (!cur_buffer_handle) { + return true; + } + // we have to scan to infinity, so we must check if we are done checking the whole file + if (!buffer_manager->Done()) { + return false; + } + // If yes, are we in the last buffer? + if (iterator.pos.buffer_idx != buffer_manager->BufferCount()) { + return false; + } + // If yes, are we in the last position? + return iterator.pos.buffer_pos + 1 == cur_buffer_handle->actual_size; +} + +void BaseScanner::Reset() { + iterator.SetCurrentPositionToBoundary(); + lines_read = 0; +} + +CSVIterator &BaseScanner::GetIterator() { + return iterator; +} + +ScannerResult &BaseScanner::ParseChunk() { + throw InternalException("ParseChunk() from CSV Base Scanner is mot implemented"); +} + +ScannerResult &BaseScanner::GetResult() { + throw InternalException("GetResult() from CSV Base Scanner is mot implemented"); +} + +void BaseScanner::Initialize() { + throw InternalException("Initialize() from CSV Base Scanner is mot implemented"); +} + +void BaseScanner::FinalizeChunkProcess() { + throw InternalException("FinalizeChunkProcess() from CSV Base Scanner is mot implemented"); +} + +CSVStateMachine &BaseScanner::GetStateMachine() { + return *state_machine; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/scanner/column_count_scanner.cpp b/src/duckdb/src/execution/operator/csv_scanner/scanner/column_count_scanner.cpp new file mode 100644 index 000000000..3623df744 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/scanner/column_count_scanner.cpp @@ -0,0 +1,98 @@ +#include "duckdb/execution/operator/csv_scanner/column_count_scanner.hpp" + +namespace duckdb { + +ColumnCountResult::ColumnCountResult(CSVStates &states, CSVStateMachine &state_machine) + : ScannerResult(states, state_machine) { +} + +void ColumnCountResult::AddValue(ColumnCountResult &result, const idx_t buffer_pos) { + result.current_column_count++; +} + +inline void ColumnCountResult::InternalAddRow() { + column_counts[result_position++] = current_column_count + 1; + current_column_count = 0; +} + +bool ColumnCountResult::AddRow(ColumnCountResult &result, const idx_t buffer_pos) { + result.InternalAddRow(); + if (!result.states.EmptyLastValue()) { + result.last_value_always_empty = false; + } + if (result.result_position >= STANDARD_VECTOR_SIZE) { + // We sniffed enough rows + return true; + } + return false; +} + +void ColumnCountResult::InvalidState(ColumnCountResult &result) { + result.result_position = 0; + result.error = true; +} + +bool ColumnCountResult::EmptyLine(ColumnCountResult &result, const idx_t buffer_pos) { + // nop + return false; +} + +void ColumnCountResult::QuotedNewLine(ColumnCountResult &result) { + // nop +} + +ColumnCountScanner::ColumnCountScanner(shared_ptr buffer_manager, + const shared_ptr &state_machine, + shared_ptr error_handler) + : BaseScanner(std::move(buffer_manager), state_machine, std::move(error_handler)), result(states, *state_machine), + column_count(1) { + sniffing = true; +} + +unique_ptr ColumnCountScanner::UpgradeToStringValueScanner() { + auto scanner = make_uniq(0, buffer_manager, state_machine, error_handler, nullptr); + scanner->sniffing = true; + return scanner; +} + +ColumnCountResult &ColumnCountScanner::ParseChunk() { + result.result_position = 0; + column_count = 1; + ParseChunkInternal(result); + return result; +} + +ColumnCountResult &ColumnCountScanner::GetResult() { + return result; +} + +void ColumnCountScanner::Initialize() { + states.Initialize(); +} + +void ColumnCountScanner::FinalizeChunkProcess() { + if (result.result_position == STANDARD_VECTOR_SIZE || result.error) { + // We are done + return; + } + // We run until we have a full chunk, or we are done scanning + while (!FinishedFile() && result.result_position < STANDARD_VECTOR_SIZE && !result.error) { + if (iterator.pos.buffer_pos == cur_buffer_handle->actual_size) { + // Move to next buffer + cur_buffer_handle = buffer_manager->GetBuffer(++iterator.pos.buffer_idx); + if (!cur_buffer_handle) { + buffer_handle_ptr = nullptr; + if (states.EmptyLine() || states.NewRow() || states.IsCurrentNewRow() || states.IsNotSet()) { + return; + } + // This means we reached the end of the file, we must add a last line if there is any to be added + result.InternalAddRow(); + return; + } + iterator.pos.buffer_pos = 0; + buffer_handle_ptr = cur_buffer_handle->Ptr(); + } + Process(result); + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/scanner/scanner_boundary.cpp b/src/duckdb/src/execution/operator/csv_scanner/scanner/scanner_boundary.cpp new file mode 100644 index 000000000..eab7125aa --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/scanner/scanner_boundary.cpp @@ -0,0 +1,109 @@ +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" + +namespace duckdb { + +CSVPosition::CSVPosition(idx_t file_idx_p, idx_t buffer_idx_p, idx_t buffer_pos_p) + : file_idx(file_idx_p), buffer_idx(buffer_idx_p), buffer_pos(buffer_pos_p) { +} +CSVPosition::CSVPosition() { +} + +CSVBoundary::CSVBoundary(idx_t file_idx_p, idx_t buffer_idx_p, idx_t buffer_pos_p, idx_t boundary_idx_p, + idx_t end_pos_p) + : file_idx(file_idx_p), buffer_idx(buffer_idx_p), buffer_pos(buffer_pos_p), boundary_idx(boundary_idx_p), + end_pos(end_pos_p) { +} +CSVBoundary::CSVBoundary() + : file_idx(0), buffer_idx(0), buffer_pos(0), boundary_idx(0), end_pos(NumericLimits::Maximum()) { +} +CSVIterator::CSVIterator(idx_t file_idx, idx_t buffer_idx, idx_t buffer_pos, idx_t boundary_idx, idx_t buffer_size) + : pos(file_idx, buffer_idx, buffer_pos), is_set(true) { + // The end of our boundary will be the buffer size itself it that's smaller than where we want to go + if (buffer_size < buffer_pos + BYTES_PER_THREAD) { + boundary = {file_idx, buffer_idx, buffer_pos, boundary_idx, buffer_size}; + } else { + boundary = {file_idx, buffer_idx, buffer_pos, boundary_idx, buffer_pos + BYTES_PER_THREAD}; + } +} + +CSVIterator::CSVIterator() : is_set(false) { +} + +void CSVBoundary::Print() { +#ifndef DUCKDB_DISABLE_PRINT + std::cout << "---Boundary: " << boundary_idx << " ---" << std::endl; + std::cout << "File Index:: " << file_idx << std::endl; + std::cout << "Buffer Index: " << buffer_idx << std::endl; + std::cout << "Buffer Pos: " << buffer_pos << std::endl; + std::cout << "End Pos: " << end_pos << std::endl; + std::cout << "------------" << end_pos << std::endl; +#endif +} + +void CSVIterator::Print() { +#ifndef DUCKDB_DISABLE_PRINT + boundary.Print(); + std::cout << "Is set: " << is_set << std::endl; +#endif +} + +bool CSVIterator::Next(CSVBufferManager &buffer_manager) { + if (!is_set) { + return false; + } + boundary.boundary_idx++; + // This is our start buffer + auto buffer = buffer_manager.GetBuffer(boundary.buffer_idx); + if (buffer->is_last_buffer && boundary.buffer_pos + CSVIterator::BYTES_PER_THREAD > buffer->actual_size) { + // 1) We are done with the current file + return false; + } else if (boundary.buffer_pos + BYTES_PER_THREAD >= buffer->actual_size) { + // 2) We still have data to scan in this file, we set the iterator accordingly. + // We must move the buffer + boundary.buffer_idx++; + boundary.buffer_pos = 0; + // Verify this buffer really exists + auto next_buffer = buffer_manager.GetBuffer(boundary.buffer_idx); + if (!next_buffer) { + return false; + } + + } else { + // 3) We are not done with the current buffer, hence we just move where we start within the buffer + boundary.buffer_pos += BYTES_PER_THREAD; + } + boundary.end_pos = boundary.buffer_pos + BYTES_PER_THREAD; + SetCurrentPositionToBoundary(); + return true; +} + +bool CSVIterator::IsBoundarySet() const { + return is_set; +} +idx_t CSVIterator::GetEndPos() const { + return boundary.end_pos; +} + +idx_t CSVIterator::GetFileIdx() const { + return pos.file_idx; +} + +idx_t CSVIterator::GetBufferIdx() const { + return boundary.buffer_idx; +} + +idx_t CSVIterator::GetBoundaryIdx() const { + return boundary.boundary_idx; +} + +void CSVIterator::SetCurrentPositionToBoundary() { + pos.file_idx = boundary.file_idx; + pos.buffer_idx = boundary.buffer_idx; + pos.buffer_pos = boundary.buffer_pos; +} + +void CSVIterator::SetStart(idx_t start) { + boundary.buffer_pos = start; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/scanner/skip_scanner.cpp b/src/duckdb/src/execution/operator/csv_scanner/scanner/skip_scanner.cpp new file mode 100644 index 000000000..486c26b9f --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/scanner/skip_scanner.cpp @@ -0,0 +1,63 @@ +#include "duckdb/execution/operator/csv_scanner/skip_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/column_count_scanner.hpp" + +namespace duckdb { + +SkipResult::SkipResult(CSVStates &states, CSVStateMachine &state_machine, idx_t rows_to_skip_p) + : ScannerResult(states, state_machine), rows_to_skip(rows_to_skip_p) { +} + +void SkipResult::AddValue(SkipResult &result, const idx_t buffer_pos) { + // nop +} + +inline void SkipResult::InternalAddRow() { + row_count++; +} + +void SkipResult::QuotedNewLine(SkipResult &result) { + // nop +} + +bool SkipResult::AddRow(SkipResult &result, const idx_t buffer_pos) { + result.InternalAddRow(); + if (result.row_count >= result.rows_to_skip) { + // We skipped enough rows + return true; + } + return false; +} + +void SkipResult::InvalidState(SkipResult &result) { + // nop +} + +bool SkipResult::EmptyLine(SkipResult &result, const idx_t buffer_pos) { + if (result.state_machine.dialect_options.num_cols == 1) { + return AddRow(result, buffer_pos); + } + return false; +} +SkipScanner::SkipScanner(shared_ptr buffer_manager, const shared_ptr &state_machine, + shared_ptr error_handler, idx_t rows_to_skip) + : BaseScanner(std::move(buffer_manager), state_machine, std::move(error_handler)), + result(states, *state_machine, rows_to_skip) { +} + +SkipResult &SkipScanner::ParseChunk() { + ParseChunkInternal(result); + return result; +} + +SkipResult &SkipScanner::GetResult() { + return result; +} + +void SkipScanner::Initialize() { + states.Initialize(); +} + +void SkipScanner::FinalizeChunkProcess() { + // nop +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/scanner/string_value_scanner.cpp b/src/duckdb/src/execution/operator/csv_scanner/scanner/string_value_scanner.cpp new file mode 100644 index 000000000..50fca884a --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/scanner/string_value_scanner.cpp @@ -0,0 +1,1063 @@ +#include "duckdb/execution/operator/csv_scanner/string_value_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_casting.hpp" +#include "duckdb/execution/operator/csv_scanner/skip_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp" +#include "duckdb/main/client_data.hpp" +#include "duckdb/common/operator/integer_cast_operator.hpp" +#include "duckdb/common/operator/double_cast_operator.hpp" +#include + +namespace duckdb { + +StringValueResult::StringValueResult(CSVStates &states, CSVStateMachine &state_machine, CSVBufferHandle &buffer_handle, + Allocator &buffer_allocator, idx_t result_size_p, idx_t buffer_position, + CSVErrorHandler &error_hander_p, CSVIterator &iterator_p, bool store_line_size_p, + shared_ptr csv_file_scan_p, idx_t &lines_read_p) + : ScannerResult(states, state_machine), number_of_columns(state_machine.dialect_options.num_cols), + null_padding(state_machine.options.null_padding), ignore_errors(state_machine.options.ignore_errors), + null_str_ptr(state_machine.options.null_str.c_str()), null_str_size(state_machine.options.null_str.size()), + result_size(result_size_p), error_handler(error_hander_p), iterator(iterator_p), + store_line_size(store_line_size_p), csv_file_scan(std::move(csv_file_scan_p)), lines_read(lines_read_p) { + // Vector information + D_ASSERT(number_of_columns > 0); + + // Buffer Information + buffer_ptr = buffer_handle.Ptr(); + buffer_size = buffer_handle.actual_size; + last_position = buffer_position; + + // Current Result information + previous_line_start = {iterator.pos.buffer_idx, iterator.pos.buffer_pos, buffer_handle.actual_size}; + pre_previous_line_start = previous_line_start; + // Fill out Parse Types + vector logical_types; + parse_types = make_unsafe_uniq_array(number_of_columns); + if (!csv_file_scan) { + for (idx_t i = 0; i < number_of_columns; i++) { + parse_types[i] = LogicalTypeId::VARCHAR; + logical_types.emplace_back(LogicalType::VARCHAR); + string name = "Column_" + to_string(i); + names.emplace_back(name); + } + } else { + if (csv_file_scan->file_types.size() > number_of_columns) { + throw InvalidInputException( + "Mismatch between the number of columns (%d) in the CSV file and what is expected in the scanner (%d).", + number_of_columns, csv_file_scan->file_types.size()); + } + for (idx_t i = 0; i < csv_file_scan->file_types.size(); i++) { + auto &type = csv_file_scan->file_types[i]; + if (StringValueScanner::CanDirectlyCast(type, state_machine.options.dialect_options.date_format)) { + parse_types[i] = type.id(); + logical_types.emplace_back(type); + } else { + parse_types[i] = LogicalTypeId::VARCHAR; + logical_types.emplace_back(LogicalType::VARCHAR); + } + } + names = csv_file_scan->names; + if (!csv_file_scan->projected_columns.empty()) { + projecting_columns = false; + projected_columns = make_unsafe_uniq_array(number_of_columns); + for (idx_t col_idx = 0; col_idx < number_of_columns; col_idx++) { + if (csv_file_scan->projected_columns.find(col_idx) == csv_file_scan->projected_columns.end()) { + // Column is not projected + projecting_columns = true; + projected_columns[col_idx] = false; + } else { + projected_columns[col_idx] = true; + } + } + } + if (!projecting_columns) { + for (idx_t j = logical_types.size(); j < number_of_columns; j++) { + // This can happen if we have sneaky null columns at the end that we wish to ignore + parse_types[j] = LogicalTypeId::VARCHAR; + logical_types.emplace_back(LogicalType::VARCHAR); + } + } + } + + // Initialize Parse Chunk + parse_chunk.Initialize(buffer_allocator, logical_types, result_size); + for (auto &col : parse_chunk.data) { + vector_ptr.push_back(FlatVector::GetData(col)); + validity_mask.push_back(&FlatVector::Validity(col)); + } +} + +void StringValueResult::AddValueToVector(const char *value_ptr, const idx_t size, bool allocate) { + if (projecting_columns) { + if (!projected_columns[cur_col_id]) { + cur_col_id++; + return; + } + } + if (size == null_str_size) { + if (((quoted && state_machine.options.allow_quoted_nulls) || !quoted)) { + bool is_null = true; + for (idx_t i = 0; i < size; i++) { + if (null_str_ptr[i] != value_ptr[i]) { + is_null = false; + break; + } + } + if (is_null) { + bool empty = false; + if (chunk_col_id < state_machine.options.force_not_null.size()) { + empty = state_machine.options.force_not_null[chunk_col_id]; + } + if (empty) { + if (chunk_col_id >= number_of_columns) { + HandleOverLimitRows(); + } + if (parse_types[chunk_col_id] != LogicalTypeId::VARCHAR) { + // If it is not a varchar, empty values are not accepted, we must error. + cast_errors[chunk_col_id] = std::string(""); + } + static_cast(vector_ptr[chunk_col_id])[number_of_rows] = string_t(); + } else { + if (chunk_col_id == number_of_columns) { + // We check for a weird case, where we ignore an extra value, if it is a null value + return; + } + validity_mask[chunk_col_id]->SetInvalid(number_of_rows); + } + cur_col_id++; + chunk_col_id++; + return; + } + } + } + if (chunk_col_id >= number_of_columns) { + HandleOverLimitRows(); + if (projecting_columns) { + if (!projected_columns[cur_col_id]) { + cur_col_id++; + return; + } + } + } + bool success = true; + switch (parse_types[chunk_col_id]) { + case LogicalTypeId::TINYINT: + success = TrySimpleIntegerCast(value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], + false); + break; + case LogicalTypeId::SMALLINT: + success = TrySimpleIntegerCast(value_ptr, size, + static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::INTEGER: + success = TrySimpleIntegerCast(value_ptr, size, + static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::BIGINT: + success = TrySimpleIntegerCast(value_ptr, size, + static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::UTINYINT: + success = TrySimpleIntegerCast( + value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::USMALLINT: + success = TrySimpleIntegerCast( + value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::UINTEGER: + success = TrySimpleIntegerCast( + value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::UBIGINT: + success = TrySimpleIntegerCast( + value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], false); + break; + case LogicalTypeId::DOUBLE: + success = + TryDoubleCast(value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], + false, state_machine.options.decimal_separator[0]); + break; + case LogicalTypeId::FLOAT: + success = TryDoubleCast(value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows], + false, state_machine.options.decimal_separator[0]); + break; + case LogicalTypeId::DATE: { + idx_t pos; + bool special; + success = Date::TryConvertDate(value_ptr, size, pos, + static_cast(vector_ptr[chunk_col_id])[number_of_rows], special, false); + break; + } + case LogicalTypeId::TIMESTAMP: { + success = Timestamp::TryConvertTimestamp( + value_ptr, size, static_cast(vector_ptr[chunk_col_id])[number_of_rows]) == + TimestampCastResult::SUCCESS; + break; + } + default: + if (allocate) { + static_cast(vector_ptr[chunk_col_id])[number_of_rows] = + StringVector::AddStringOrBlob(parse_chunk.data[chunk_col_id], string_t(value_ptr, size)); + } else { + static_cast(vector_ptr[chunk_col_id])[number_of_rows] = string_t(value_ptr, size); + } + break; + } + if (!success) { + // We had a casting error, we push it here because we can only error when finishing the line read. + cast_errors[cur_col_id] = std::string(value_ptr, size); + } + cur_col_id++; + chunk_col_id++; +} + +Value StringValueResult::GetValue(idx_t row_idx, idx_t col_idx) { + if (validity_mask[col_idx]->AllValid()) { + return Value(static_cast(vector_ptr[col_idx])[row_idx]); + } else { + if (validity_mask[col_idx]->RowIsValid(row_idx)) { + return Value(static_cast(vector_ptr[col_idx])[row_idx]); + } else { + return Value(); + } + } +} +DataChunk &StringValueResult::ToChunk() { + parse_chunk.SetCardinality(number_of_rows); + return parse_chunk; +} + +void StringValueResult::AddQuotedValue(StringValueResult &result, const idx_t buffer_pos) { + if (result.escaped) { + if (result.projecting_columns) { + if (!result.projected_columns[result.cur_col_id]) { + result.cur_col_id++; + return; + } + } + // If it's an escaped value we have to remove all the escapes, this is not really great + auto value = StringValueScanner::RemoveEscape( + result.buffer_ptr + result.quoted_position + 1, buffer_pos - result.quoted_position - 2, + result.state_machine.options.GetEscape()[0], result.parse_chunk.data[result.chunk_col_id]); + result.AddValueToVector(value.GetData(), value.GetSize()); + } else { + if (buffer_pos < result.last_position + 2) { + // empty value + auto value = string_t(); + result.AddValueToVector(value.GetData(), value.GetSize()); + } else { + result.AddValueToVector(result.buffer_ptr + result.quoted_position + 1, + buffer_pos - result.quoted_position - 2); + } + } + result.quoted = false; + result.escaped = false; +} + +void StringValueResult::AddValue(StringValueResult &result, const idx_t buffer_pos) { + if (result.last_position > buffer_pos) { + return; + } + if (result.quoted) { + StringValueResult::AddQuotedValue(result, buffer_pos); + } else { + result.AddValueToVector(result.buffer_ptr + result.last_position, buffer_pos - result.last_position); + } + result.last_position = buffer_pos + 1; +} + +void StringValueResult::HandleOverLimitRows() { + auto csv_error = + CSVError::IncorrectColumnAmountError(state_machine.options, nullptr, number_of_columns, cur_col_id + 1); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), number_of_rows + 1); + error_handler.Error(lines_per_batch, csv_error); + // If we get here we need to remove the last line + cur_col_id = 0; + chunk_col_id = 0; +} + +void StringValueResult::QuotedNewLine(StringValueResult &result) { + result.quoted_new_line = true; +} + +void StringValueResult::NullPaddingQuotedNewlineCheck() { + // We do some checks for null_padding correctness + if (state_machine.options.null_padding && iterator.IsBoundarySet() && quoted_new_line && iterator.done) { + // If we have null_padding set, we found a quoted new line, we are scanning the file in parallel and it's the + // last row of this thread. + auto csv_error = CSVError::NullPaddingFail(state_machine.options); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), number_of_rows + 1); + error_handler.Error(lines_per_batch, csv_error, true); + } +} + +bool StringValueResult::AddRowInternal() { + if (!cast_errors.empty()) { + // A wild casting error appears + // Recreate row for rejects-table + vector row; + if (!state_machine.options.rejects_table_name.empty()) { + for (idx_t col = 0; col < parse_chunk.ColumnCount(); col++) { + if (cast_errors.find(col) != cast_errors.end()) { + row.push_back(cast_errors[col]); + } else { + row.push_back(parse_chunk.data[col].GetValue(number_of_rows)); + } + } + } + for (auto &cast_error : cast_errors) { + std::ostringstream error; + // Casting Error Message + error << "Could not convert string \"" << cast_error.second << "\" to \'" + << LogicalTypeIdToString(parse_types[cast_error.first]) << "\'"; + auto error_string = error.str(); + auto csv_error = CSVError::CastError(state_machine.options, names[cast_error.first], error_string, + cast_error.first, row); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), lines_read - 1); + error_handler.Error(lines_per_batch, csv_error); + } + // If we got here it means we are ignoring errors, hence we need to signify to our result scanner to ignore this + // row + // Cleanup this line and continue + cast_errors.clear(); + cur_col_id = 0; + chunk_col_id = 0; + return false; + } + NullPaddingQuotedNewlineCheck(); + quoted_new_line = false; + // We need to check if we are getting the correct number of columns here. + // If columns are correct, we add it, and that's it. + if (cur_col_id != number_of_columns) { + // We have too few columns: + if (null_padding) { + while (cur_col_id < number_of_columns) { + bool empty = false; + if (cur_col_id < state_machine.options.force_not_null.size()) { + empty = state_machine.options.force_not_null[cur_col_id]; + } + if (empty) { + static_cast(vector_ptr[chunk_col_id])[number_of_rows] = string_t(); + } else { + validity_mask[chunk_col_id]->SetInvalid(number_of_rows); + } + cur_col_id++; + chunk_col_id++; + } + } else { + // If we are not nullpadding this is an error + auto csv_error = + CSVError::IncorrectColumnAmountError(state_machine.options, nullptr, number_of_columns, cur_col_id); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), number_of_rows + 1); + error_handler.Error(lines_per_batch, csv_error); + // If we are here we ignore_errors, so we delete this line + number_of_rows--; + } + } + cur_col_id = 0; + chunk_col_id = 0; + number_of_rows++; + if (number_of_rows >= result_size) { + // We have a full chunk + return true; + } + return false; +} + +bool StringValueResult::AddRow(StringValueResult &result, const idx_t buffer_pos) { + if (result.last_position <= buffer_pos) { + LinePosition current_line_start = {result.iterator.pos.buffer_idx, result.iterator.pos.buffer_pos, + result.buffer_size}; + idx_t current_line_size = current_line_start - result.previous_line_start; + if (result.store_line_size) { + result.error_handler.NewMaxLineSize(current_line_size); + } + if (current_line_size > result.state_machine.options.maximum_line_size) { + auto csv_error = CSVError::LineSizeError(result.state_machine.options, current_line_size); + LinesPerBoundary lines_per_batch(result.iterator.GetBoundaryIdx(), result.number_of_rows + 1); + result.error_handler.Error(lines_per_batch, csv_error); + } + result.pre_previous_line_start = result.previous_line_start; + result.previous_line_start = current_line_start; + // We add the value + if (result.quoted) { + StringValueResult::AddQuotedValue(result, buffer_pos); + } else { + result.AddValueToVector(result.buffer_ptr + result.last_position, buffer_pos - result.last_position); + } + if (result.state_machine.dialect_options.state_machine_options.new_line == NewLineIdentifier::CARRY_ON) { + if (result.states.states[1] == CSVState::RECORD_SEPARATOR) { + // Even though this is marked as a carry on, this is a hippie mixie + result.last_position = buffer_pos + 1; + } else { + result.last_position = buffer_pos + 2; + } + } else { + result.last_position = buffer_pos + 1; + } + } + + // We add the value + return result.AddRowInternal(); +} + +void StringValueResult::InvalidState(StringValueResult &result) { + // FIXME: How do we recover from an invalid state? Can we restart the state machine and jump to the next row? + auto csv_error = CSVError::UnterminatedQuotesError(result.state_machine.options, + static_cast(result.vector_ptr[result.chunk_col_id]), + result.number_of_rows, result.cur_col_id); + LinesPerBoundary lines_per_batch(result.iterator.GetBoundaryIdx(), result.number_of_rows); + result.error_handler.Error(lines_per_batch, csv_error); +} + +bool StringValueResult::EmptyLine(StringValueResult &result, const idx_t buffer_pos) { + // We care about empty lines if this is a single column csv file + result.last_position = buffer_pos + 1; + if (result.states.IsCarriageReturn() && + result.state_machine.dialect_options.state_machine_options.new_line == NewLineIdentifier::CARRY_ON) { + result.last_position++; + } + if (result.number_of_columns == 1) { + if (result.null_str_size == 0) { + bool empty = false; + if (!result.state_machine.options.force_not_null.empty()) { + empty = result.state_machine.options.force_not_null[0]; + } + if (empty) { + static_cast(result.vector_ptr[0])[result.number_of_rows] = string_t(); + } else { + result.validity_mask[0]->SetInvalid(result.number_of_rows); + } + result.number_of_rows++; + } + if (result.number_of_rows >= result.result_size) { + // We have a full chunk + return true; + } + } + return false; +} + +StringValueScanner::StringValueScanner(idx_t scanner_idx_p, const shared_ptr &buffer_manager, + const shared_ptr &state_machine, + const shared_ptr &error_handler, + const shared_ptr &csv_file_scan, CSVIterator boundary, + idx_t result_size) + : BaseScanner(buffer_manager, state_machine, error_handler, csv_file_scan, boundary), scanner_idx(scanner_idx_p), + result(states, *state_machine, *cur_buffer_handle, BufferAllocator::Get(buffer_manager->context), result_size, + iterator.pos.buffer_pos, *error_handler, iterator, + buffer_manager->context.client_data->debug_set_max_line_length, csv_file_scan, lines_read) { +} + +StringValueScanner::StringValueScanner(const shared_ptr &buffer_manager, + const shared_ptr &state_machine, + const shared_ptr &error_handler) + : BaseScanner(buffer_manager, state_machine, error_handler, nullptr, {}), scanner_idx(0), + result(states, *state_machine, *cur_buffer_handle, Allocator::DefaultAllocator(), STANDARD_VECTOR_SIZE, + iterator.pos.buffer_pos, *error_handler, iterator, + buffer_manager->context.client_data->debug_set_max_line_length, csv_file_scan, lines_read) { +} + +unique_ptr StringValueScanner::GetCSVScanner(ClientContext &context, CSVReaderOptions &options) { + auto state_machine = make_shared(options, options.dialect_options.state_machine_options, + CSVStateMachineCache::Get(context)); + + state_machine->dialect_options.num_cols = options.dialect_options.num_cols; + state_machine->dialect_options.header = options.dialect_options.header; + auto buffer_manager = make_shared(context, options, options.file_path, 0); + auto scanner = make_uniq(buffer_manager, state_machine, make_shared()); + scanner->csv_file_scan = make_shared(context, options.file_path, options); + scanner->csv_file_scan->InitializeProjection(); + return scanner; +} + +bool StringValueScanner::FinishedIterator() { + return iterator.done; +} + +StringValueResult &StringValueScanner::ParseChunk() { + result.number_of_rows = 0; + result.cur_col_id = 0; + result.chunk_col_id = 0; + for (auto &v : result.validity_mask) { + v->SetAllValid(result.result_size); + } + ParseChunkInternal(result); + return result; +} + +void StringValueScanner::Flush(DataChunk &insert_chunk) { + auto &process_result = ParseChunk(); + // First Get Parsed Chunk + auto &parse_chunk = process_result.ToChunk(); + + if (parse_chunk.size() == 0) { + return; + } + // convert the columns in the parsed chunk to the types of the table + insert_chunk.SetCardinality(parse_chunk); + + // We keep track of the borked lines, in case we are ignoring errors + unordered_set borked_lines; + D_ASSERT(csv_file_scan); + + auto &reader_data = csv_file_scan->reader_data; + // Now Do the cast-aroo + for (idx_t c = 0; c < reader_data.column_ids.size(); c++) { + idx_t col_idx = c; + idx_t result_idx = reader_data.column_mapping[c]; + if (!csv_file_scan->projection_ids.empty()) { + result_idx = reader_data.column_mapping[csv_file_scan->projection_ids[c].second]; + } + if (col_idx >= parse_chunk.ColumnCount()) { + throw InvalidInputException("Mismatch between the schema of different files"); + } + auto &parse_vector = parse_chunk.data[col_idx]; + auto &result_vector = insert_chunk.data[result_idx]; + auto &type = result_vector.GetType(); + auto &parse_type = parse_vector.GetType(); + if (type == LogicalType::VARCHAR || (type != LogicalType::VARCHAR && parse_type != LogicalType::VARCHAR)) { + // reinterpret rather than reference + result_vector.Reinterpret(parse_vector); + } else { + string error_message; + bool success; + idx_t line_error = 0; + bool line_error_set = true; + + if (!state_machine->options.dialect_options.date_format.at(LogicalTypeId::DATE).GetValue().Empty() && + type.id() == LogicalTypeId::DATE) { + // use the date format to cast the chunk + success = CSVCast::TryCastDateVector(state_machine->options.dialect_options.date_format, parse_vector, + result_vector, parse_chunk.size(), error_message, line_error); + } else if (!state_machine->options.dialect_options.date_format.at(LogicalTypeId::TIMESTAMP) + .GetValue() + .Empty() && + type.id() == LogicalTypeId::TIMESTAMP) { + // use the date format to cast the chunk + success = + CSVCast::TryCastTimestampVector(state_machine->options.dialect_options.date_format, parse_vector, + result_vector, parse_chunk.size(), error_message); + } else if (state_machine->options.decimal_separator != "." && + (type.id() == LogicalTypeId::FLOAT || type.id() == LogicalTypeId::DOUBLE)) { + success = + CSVCast::TryCastFloatingVectorCommaSeparated(state_machine->options, parse_vector, result_vector, + parse_chunk.size(), error_message, type, line_error); + } else if (state_machine->options.decimal_separator != "." && type.id() == LogicalTypeId::DECIMAL) { + success = CSVCast::TryCastDecimalVectorCommaSeparated( + state_machine->options, parse_vector, result_vector, parse_chunk.size(), error_message, type); + } else { + // target type is not varchar: perform a cast + success = VectorOperations::TryCast(buffer_manager->context, parse_vector, result_vector, + parse_chunk.size(), &error_message); + line_error_set = false; + } + if (success) { + continue; + } + // An error happened, to propagate it we need to figure out the exact line where the casting failed. + UnifiedVectorFormat inserted_column_data; + result_vector.ToUnifiedFormat(parse_chunk.size(), inserted_column_data); + UnifiedVectorFormat parse_column_data; + parse_vector.ToUnifiedFormat(parse_chunk.size(), parse_column_data); + if (!line_error_set) { + for (; line_error < parse_chunk.size(); line_error++) { + if (!inserted_column_data.validity.RowIsValid(line_error) && + parse_column_data.validity.RowIsValid(line_error)) { + break; + } + } + } + { + vector row; + for (idx_t col = 0; col < parse_chunk.ColumnCount(); col++) { + row.push_back(parse_chunk.GetValue(col, line_error)); + } + auto csv_error = CSVError::CastError(state_machine->options, csv_file_scan->names[col_idx], + error_message, col_idx, row); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), + lines_read - parse_chunk.size() + line_error); + error_handler->Error(lines_per_batch, csv_error); + } + borked_lines.insert(line_error++); + D_ASSERT(state_machine->options.ignore_errors); + // We are ignoring errors. We must continue but ignoring borked rows + for (; line_error < parse_chunk.size(); line_error++) { + if (!inserted_column_data.validity.RowIsValid(line_error) && + parse_column_data.validity.RowIsValid(line_error)) { + borked_lines.insert(line_error); + vector row; + for (idx_t col = 0; col < parse_chunk.ColumnCount(); col++) { + row.push_back(parse_chunk.GetValue(col, line_error)); + } + auto csv_error = CSVError::CastError(state_machine->options, csv_file_scan->names[col_idx], + error_message, col_idx, row); + LinesPerBoundary lines_per_batch(iterator.GetBoundaryIdx(), + lines_read - parse_chunk.size() + line_error); + error_handler->Error(lines_per_batch, csv_error); + } + } + } + } + if (!borked_lines.empty()) { + // We must remove the borked lines from our chunk + SelectionVector succesful_rows(parse_chunk.size() - borked_lines.size()); + idx_t sel_idx = 0; + for (idx_t row_idx = 0; row_idx < parse_chunk.size(); row_idx++) { + if (borked_lines.find(row_idx) == borked_lines.end()) { + succesful_rows.set_index(sel_idx++, row_idx); + } + } + // Now we slice the result + insert_chunk.Slice(succesful_rows, sel_idx); + } +} + +void StringValueScanner::Initialize() { + states.Initialize(); + + if (result.result_size != 1 && !(sniffing && state_machine->options.null_padding && + !state_machine->options.dialect_options.skip_rows.IsSetByUser())) { + SetStart(); + } + result.last_position = iterator.pos.buffer_pos; + result.previous_line_start = {iterator.pos.buffer_idx, iterator.pos.buffer_pos, cur_buffer_handle->actual_size}; + + result.pre_previous_line_start = result.previous_line_start; +} + +void StringValueScanner::ProcessExtraRow() { + result.NullPaddingQuotedNewlineCheck(); + idx_t to_pos = cur_buffer_handle->actual_size; + while (iterator.pos.buffer_pos < to_pos) { + state_machine->Transition(states, buffer_handle_ptr[iterator.pos.buffer_pos]); + switch (states.states[1]) { + case CSVState::INVALID: + result.InvalidState(result); + iterator.pos.buffer_pos++; + return; + case CSVState::RECORD_SEPARATOR: + if (states.states[0] == CSVState::RECORD_SEPARATOR) { + lines_read++; + result.EmptyLine(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + return; + } else if (states.states[0] != CSVState::CARRIAGE_RETURN) { + lines_read++; + result.AddRow(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + return; + } + iterator.pos.buffer_pos++; + break; + case CSVState::CARRIAGE_RETURN: + lines_read++; + if (states.states[0] != CSVState::RECORD_SEPARATOR) { + result.AddRow(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + return; + } else { + result.EmptyLine(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + return; + } + case CSVState::DELIMITER: + result.AddValue(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + break; + case CSVState::QUOTED: + if (states.states[0] == CSVState::UNQUOTED) { + result.SetEscaped(result); + } + result.SetQuoted(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + while (state_machine->transition_array + .skip_quoted[static_cast(buffer_handle_ptr[iterator.pos.buffer_pos])] && + iterator.pos.buffer_pos < to_pos - 1) { + iterator.pos.buffer_pos++; + } + break; + case CSVState::ESCAPE: + result.SetEscaped(result); + iterator.pos.buffer_pos++; + break; + case CSVState::STANDARD: + iterator.pos.buffer_pos++; + while (state_machine->transition_array + .skip_standard[static_cast(buffer_handle_ptr[iterator.pos.buffer_pos])] && + iterator.pos.buffer_pos < to_pos - 1) { + iterator.pos.buffer_pos++; + } + break; + case CSVState::QUOTED_NEW_LINE: + result.quoted_new_line = true; + result.NullPaddingQuotedNewlineCheck(); + iterator.pos.buffer_pos++; + break; + default: + iterator.pos.buffer_pos++; + break; + } + } +} + +string_t StringValueScanner::RemoveEscape(const char *str_ptr, idx_t end, char escape, Vector &vector) { + // Figure out the exact size + idx_t str_pos = 0; + bool just_escaped = false; + for (idx_t cur_pos = 0; cur_pos < end; cur_pos++) { + if (str_ptr[cur_pos] == escape && !just_escaped) { + just_escaped = true; + } else { + just_escaped = false; + str_pos++; + } + } + + auto removed_escapes = StringVector::EmptyString(vector, str_pos); + auto removed_escapes_ptr = removed_escapes.GetDataWriteable(); + // Allocate string and copy it + str_pos = 0; + just_escaped = false; + for (idx_t cur_pos = 0; cur_pos < end; cur_pos++) { + char c = str_ptr[cur_pos]; + if (c == escape && !just_escaped) { + just_escaped = true; + } else { + just_escaped = false; + removed_escapes_ptr[str_pos++] = c; + } + } + removed_escapes.Finalize(); + return removed_escapes; +} + +void StringValueScanner::ProcessOverbufferValue() { + // Process first string + states.Initialize(); + string overbuffer_string; + auto previous_buffer = previous_buffer_handle->Ptr(); + if (result.last_position == previous_buffer_handle->actual_size) { + state_machine->Transition(states, previous_buffer[result.last_position - 1]); + } + idx_t j = 0; + result.quoted = false; + for (idx_t i = result.last_position; i < previous_buffer_handle->actual_size; i++) { + state_machine->Transition(states, previous_buffer[i]); + if (states.EmptyLine() || states.IsCurrentNewRow()) { + continue; + } + if (states.NewRow() || states.NewValue()) { + break; + } else { + overbuffer_string += previous_buffer[i]; + } + if (states.IsQuoted()) { + result.SetQuoted(result, j); + } + if (states.IsEscaped()) { + result.escaped = true; + } + j++; + } + if (overbuffer_string.empty() && + state_machine->dialect_options.state_machine_options.new_line == NewLineIdentifier::CARRY_ON) { + if (buffer_handle_ptr[iterator.pos.buffer_pos] == '\n') { + iterator.pos.buffer_pos++; + } + } + // second buffer + for (; iterator.pos.buffer_pos < cur_buffer_handle->actual_size; iterator.pos.buffer_pos++) { + state_machine->Transition(states, buffer_handle_ptr[iterator.pos.buffer_pos]); + if (states.EmptyLine()) { + if (state_machine->dialect_options.num_cols == 1) { + break; + } else { + continue; + } + } + if (states.NewRow() || states.NewValue()) { + break; + } else { + overbuffer_string += buffer_handle_ptr[iterator.pos.buffer_pos]; + } + if (states.IsQuoted()) { + result.SetQuoted(result, j); + } + if (states.IsEscaped()) { + result.escaped = true; + } + j++; + } + string_t value; + if (result.quoted) { + value = string_t(overbuffer_string.c_str() + result.quoted_position, + overbuffer_string.size() - 1 - result.quoted_position); + if (result.escaped) { + const auto str_ptr = static_cast(overbuffer_string.c_str() + result.quoted_position); + value = + StringValueScanner::RemoveEscape(str_ptr, overbuffer_string.size() - 2, + state_machine->dialect_options.state_machine_options.escape.GetValue(), + result.parse_chunk.data[result.chunk_col_id]); + } + } else { + value = string_t(overbuffer_string.c_str(), overbuffer_string.size()); + } + + if (states.EmptyLine() && state_machine->dialect_options.num_cols == 1) { + result.EmptyLine(result, iterator.pos.buffer_pos); + } else if (!states.IsNotSet()) { + result.AddValueToVector(value.GetData(), value.GetSize(), true); + } + + if (states.NewRow() && !states.IsNotSet()) { + result.AddRowInternal(); + lines_read++; + } + + if (iterator.pos.buffer_pos >= cur_buffer_handle->actual_size && cur_buffer_handle->is_last_buffer) { + result.added_last_line = true; + } + if (states.IsCarriageReturn() && + state_machine->dialect_options.state_machine_options.new_line == NewLineIdentifier::CARRY_ON) { + result.last_position = ++iterator.pos.buffer_pos + 1; + } else { + result.last_position = ++iterator.pos.buffer_pos; + } + // Be sure to reset the quoted and escaped variables + result.quoted = false; + result.escaped = false; +} + +bool StringValueScanner::MoveToNextBuffer() { + if (iterator.pos.buffer_pos >= cur_buffer_handle->actual_size) { + previous_buffer_handle = std::move(cur_buffer_handle); + cur_buffer_handle = buffer_manager->GetBuffer(++iterator.pos.buffer_idx); + if (!cur_buffer_handle) { + iterator.pos.buffer_idx--; + buffer_handle_ptr = nullptr; + // We do not care if it's a quoted new line on the last row of our file. + result.quoted_new_line = false; + // This means we reached the end of the file, we must add a last line if there is any to be added + if (states.EmptyLine() || states.NewRow() || result.added_last_line || states.IsCurrentNewRow() || + states.IsNotSet()) { + if (result.cur_col_id == result.number_of_columns) { + result.number_of_rows++; + } + result.cur_col_id = 0; + result.chunk_col_id = 0; + return false; + } else if (states.NewValue()) { + lines_read++; + // we add the value + result.AddValue(result, previous_buffer_handle->actual_size); + // And an extra empty value to represent what comes after the delimiter + result.AddRow(result, previous_buffer_handle->actual_size); + } else if (states.IsQuotedCurrent()) { + // Unterminated quote + result.InvalidState(result); + } else { + lines_read++; + result.AddRow(result, previous_buffer_handle->actual_size); + } + return false; + } + iterator.pos.buffer_pos = 0; + buffer_handle_ptr = cur_buffer_handle->Ptr(); + // Handle overbuffer value + ProcessOverbufferValue(); + result.buffer_ptr = buffer_handle_ptr; + result.buffer_size = cur_buffer_handle->actual_size; + return true; + } + return false; +} + +void StringValueScanner::SkipBOM() { + if (cur_buffer_handle->actual_size >= 3 && result.buffer_ptr[0] == '\xEF' && result.buffer_ptr[1] == '\xBB' && + result.buffer_ptr[2] == '\xBF') { + iterator.pos.buffer_pos = 3; + } +} + +void StringValueScanner::SkipCSVRows() { + idx_t rows_to_skip = + state_machine->dialect_options.skip_rows.GetValue() + state_machine->dialect_options.header.GetValue(); + if (rows_to_skip == 0) { + return; + } + SkipScanner row_skipper(buffer_manager, state_machine, error_handler, rows_to_skip); + row_skipper.ParseChunk(); + iterator.pos.buffer_pos = row_skipper.GetIteratorPosition(); + if (row_skipper.state_machine->options.dialect_options.state_machine_options.new_line == + NewLineIdentifier::CARRY_ON && + row_skipper.states.states[1] == CSVState::CARRIAGE_RETURN) { + iterator.pos.buffer_pos++; + } + if (result.store_line_size) { + result.error_handler.NewMaxLineSize(iterator.pos.buffer_pos); + } + lines_read += row_skipper.GetLinesRead(); +} + +void StringValueScanner::SkipUntilNewLine() { + // Now skip until next newline + if (state_machine->options.dialect_options.state_machine_options.new_line.GetValue() == + NewLineIdentifier::CARRY_ON) { + bool carriage_return = false; + for (; iterator.pos.buffer_pos < cur_buffer_handle->actual_size; iterator.pos.buffer_pos++) { + if (buffer_handle_ptr[iterator.pos.buffer_pos] == '\r') { + carriage_return = true; + } + if (buffer_handle_ptr[iterator.pos.buffer_pos] == '\n') { + if (carriage_return) { + iterator.pos.buffer_pos++; + return; + } + } + } + } else { + for (; iterator.pos.buffer_pos < cur_buffer_handle->actual_size; iterator.pos.buffer_pos++) { + if (buffer_handle_ptr[iterator.pos.buffer_pos] == '\n' || + buffer_handle_ptr[iterator.pos.buffer_pos] == '\r') { + iterator.pos.buffer_pos++; + return; + } + } + } +} + +bool StringValueScanner::CanDirectlyCast(const LogicalType &type, + const map> &format_options) { + + switch (type.id()) { + // All Integers (Except HugeInt) + case LogicalTypeId::TINYINT: + case LogicalTypeId::SMALLINT: + case LogicalTypeId::INTEGER: + case LogicalTypeId::BIGINT: + case LogicalTypeId::UTINYINT: + case LogicalTypeId::USMALLINT: + case LogicalTypeId::UINTEGER: + case LogicalTypeId::UBIGINT: + case LogicalTypeId::DOUBLE: + case LogicalTypeId::FLOAT: + return true; + case LogicalTypeId::DATE: + // We can only internally cast YYYY-MM-DD + if (format_options.at(LogicalTypeId::DATE).GetValue().format_specifier == "%Y-%m-%d") { + return true; + } else { + return false; + } + case LogicalTypeId::TIMESTAMP: + if (format_options.at(LogicalTypeId::TIMESTAMP).GetValue().format_specifier == "%Y-%m-%d %H:%M:%S") { + return true; + } else { + return false; + } + case LogicalType::VARCHAR: + return true; + default: + return false; + } +} + +void StringValueScanner::SetStart() { + if (iterator.pos.buffer_idx == 0 && iterator.pos.buffer_pos == 0) { + // This means this is the very first buffer + // This CSV is not from auto-detect, so we don't know where exactly it starts + // Hence we potentially have to skip empty lines and headers. + SkipBOM(); + SkipCSVRows(); + return; + } + // We have to look for a new line that fits our schema + // 1. We walk until the next new line + bool line_found; + unique_ptr scan_finder; + do { + SkipUntilNewLine(); + if (state_machine->options.null_padding) { + // When Null Padding, we assume we start from the correct new-line + return; + } + scan_finder = make_uniq(0, buffer_manager, state_machine, + make_shared(true), csv_file_scan, iterator, 1); + auto &tuples = scan_finder->ParseChunk(); + line_found = true; + if (tuples.number_of_rows != 1) { + line_found = false; + // If no tuples were parsed, this is not the correct start, we need to skip until the next new line + // Or if columns don't match, this is not the correct start, we need to skip until the next new line + if (scan_finder->previous_buffer_handle) { + if (scan_finder->iterator.pos.buffer_pos >= scan_finder->previous_buffer_handle->actual_size && + scan_finder->previous_buffer_handle->is_last_buffer) { + iterator.pos.buffer_idx = scan_finder->iterator.pos.buffer_idx; + iterator.pos.buffer_pos = scan_finder->iterator.pos.buffer_pos; + result.last_position = iterator.pos.buffer_pos; + iterator.done = scan_finder->iterator.done; + return; + } + } + } + } while (!line_found); + iterator.pos.buffer_idx = scan_finder->result.pre_previous_line_start.buffer_idx; + iterator.pos.buffer_pos = scan_finder->result.pre_previous_line_start.buffer_pos; + result.last_position = iterator.pos.buffer_pos; +} + +void StringValueScanner::FinalizeChunkProcess() { + if (result.number_of_rows >= result.result_size || iterator.done) { + // We are done + if (!sniffing) { + if (csv_file_scan) { + csv_file_scan->bytes_read += bytes_read; + bytes_read = 0; + } + } + return; + } + // If we are not done we have two options. + // 1) If a boundary is set. + if (iterator.IsBoundarySet()) { + iterator.done = true; + // We read until the next line or until we have nothing else to read. + // Move to next buffer + if (!cur_buffer_handle) { + return; + } + bool moved = MoveToNextBuffer(); + if (cur_buffer_handle) { + if (moved && result.cur_col_id < result.number_of_columns && result.cur_col_id > 0) { + ProcessExtraRow(); + } else if (!moved) { + ProcessExtraRow(); + } + if (cur_buffer_handle->is_last_buffer && iterator.pos.buffer_pos >= cur_buffer_handle->actual_size) { + MoveToNextBuffer(); + } + } + } else { + // 2) If a boundary is not set + // We read until the chunk is complete, or we have nothing else to read. + while (!FinishedFile() && result.number_of_rows < result.result_size) { + MoveToNextBuffer(); + if (result.number_of_rows >= result.result_size) { + return; + } + if (cur_buffer_handle) { + Process(result); + } + } + iterator.done = FinishedFile(); + if (result.null_padding) { + while (result.cur_col_id < result.number_of_columns) { + result.validity_mask[result.chunk_col_id++]->SetInvalid(result.number_of_rows); + result.cur_col_id++; + } + result.number_of_rows++; + } + } +} +} // namespace duckdb 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 718050431..2477ad67c 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 @@ -1,6 +1,4 @@ -#include - -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" namespace duckdb { @@ -15,6 +13,8 @@ CSVSniffer::CSVSniffer(CSVReaderOptions &options_p, shared_ptr } // Initialize max columns found to either 0 or however many were set max_columns_found = set_columns.Size(); + error_handler = make_shared(options.ignore_errors); + detection_error_handler = make_shared(true); } bool SetColumns::IsSet() { @@ -44,31 +44,42 @@ void MatchAndReplace(CSVOption &original, CSVOption &sniffed, const string original.Set(sniffed.GetValue(), false); } } -void MatchAndRepaceUserSetVariables(DialectOptions &original, DialectOptions &sniffed, string &error) { +void MatchAndRepaceUserSetVariables(DialectOptions &original, DialectOptions &sniffed, string &error, bool found_date, + bool found_timestamp) { MatchAndReplace(original.header, sniffed.header, "Header", error); - if (sniffed.new_line.GetValue() != NewLineIdentifier::NOT_SET) { + if (sniffed.state_machine_options.new_line.GetValue() != NewLineIdentifier::NOT_SET) { // Is sniffed line is not set (e.g., single-line file) , we don't try to replace and match. - MatchAndReplace(original.new_line, sniffed.new_line, "New Line", error); + MatchAndReplace(original.state_machine_options.new_line, sniffed.state_machine_options.new_line, "New Line", + error); } MatchAndReplace(original.skip_rows, sniffed.skip_rows, "Skip Rows", error); MatchAndReplace(original.state_machine_options.delimiter, sniffed.state_machine_options.delimiter, "Delimiter", error); MatchAndReplace(original.state_machine_options.quote, sniffed.state_machine_options.quote, "Quote", error); MatchAndReplace(original.state_machine_options.escape, sniffed.state_machine_options.escape, "Escape", error); - MatchAndReplace(original.date_format[LogicalTypeId::DATE], sniffed.date_format[LogicalTypeId::DATE], "Date Format", - error); - MatchAndReplace(original.date_format[LogicalTypeId::TIMESTAMP], sniffed.date_format[LogicalTypeId::TIMESTAMP], - "Timestamp Format", error); + if (found_date) { + MatchAndReplace(original.date_format[LogicalTypeId::DATE], sniffed.date_format[LogicalTypeId::DATE], + "Date Format", error); + } + if (found_timestamp) { + MatchAndReplace(original.date_format[LogicalTypeId::TIMESTAMP], sniffed.date_format[LogicalTypeId::TIMESTAMP], + "Timestamp Format", error); + } } // Set the CSV Options in the reference void CSVSniffer::SetResultOptions() { - MatchAndRepaceUserSetVariables(options.dialect_options, best_candidate->dialect_options, - options.sniffer_user_mismatch_error); - if (options.dialect_options.header.GetValue()) { - options.dialect_options.true_start = best_start_with_header; - } else { - options.dialect_options.true_start = best_start_without_header; + bool found_date = false; + bool found_timestamp = false; + for (auto &type : detected_types) { + if (type == LogicalType::DATE) { + found_date = true; + } else if (type == LogicalType::TIMESTAMP) { + found_timestamp = true; + } } + MatchAndRepaceUserSetVariables(options.dialect_options, best_candidate->GetStateMachine().dialect_options, + options.sniffer_user_mismatch_error, found_date, found_timestamp); + options.dialect_options.num_cols = best_candidate->GetStateMachine().dialect_options.num_cols; } SnifferResult CSVSniffer::SniffCSV(bool force_match) { @@ -82,9 +93,20 @@ SnifferResult CSVSniffer::SniffCSV(bool force_match) { DetectHeader(); // 5. Type Replacement ReplaceTypes(); + if (!best_candidate->error_handler->errors.empty() && !options.ignore_errors) { + for (auto &error : best_candidate->error_handler->errors) { + if (error.second.type == CSVErrorType::MAXIMUM_LINE_SIZE) { + // If it's a maximul line size error, we can do it now. + error_handler->Error(error.second); + } + } + auto error = CSVError::SniffingError(options.file_path); + error_handler->Error(error); + } D_ASSERT(best_sql_types_candidates_per_column_idx.size() == names.size()); // We are done, Set the CSV Options in the reference. Construct and return the result. SetResultOptions(); + options.auto_detect = true; // Check if everything matches auto &error = options.sniffer_user_mismatch_error; if (set_columns.IsSet()) { 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 39b41c72e..d00d0972d 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 @@ -1,86 +1,8 @@ -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" #include "duckdb/main/client_data.hpp" namespace duckdb { -struct SniffDialect { - inline static void Initialize(CSVStateMachine &machine) { - machine.state = CSVState::EMPTY_LINE; - machine.previous_state = CSVState::EMPTY_LINE; - machine.pre_previous_state = CSVState::EMPTY_LINE; - machine.cur_rows = 0; - machine.column_count = 1; - } - - inline static bool Process(CSVStateMachine &machine, vector &sniffed_column_counts, char current_char, - idx_t current_pos) { - - D_ASSERT(sniffed_column_counts.size() == STANDARD_VECTOR_SIZE); - - if (machine.state == CSVState::INVALID) { - sniffed_column_counts.clear(); - return true; - } - machine.Transition(current_char); - - bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; - machine.column_count += machine.previous_state == CSVState::DELIMITER; - sniffed_column_counts[machine.cur_rows] = machine.column_count; - machine.cur_rows += machine.previous_state == CSVState::RECORD_SEPARATOR; - machine.column_count -= (machine.column_count - 1) * (machine.previous_state == CSVState::RECORD_SEPARATOR); - - // It means our carriage return is actually a record separator - machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return; - machine.column_count -= - (machine.column_count - 1) * (machine.state != CSVState::RECORD_SEPARATOR && carriage_return); - - // Identify what is our line separator - machine.carry_on_separator = - (machine.state == CSVState::RECORD_SEPARATOR && carriage_return) || machine.carry_on_separator; - machine.single_record_separator = ((machine.state != CSVState::RECORD_SEPARATOR && carriage_return) || - (machine.state == CSVState::RECORD_SEPARATOR && !carriage_return)) || - machine.single_record_separator; - if (machine.cur_rows >= STANDARD_VECTOR_SIZE) { - // We sniffed enough rows - return true; - } - return false; - } - inline static void Finalize(CSVStateMachine &machine, vector &sniffed_column_counts) { - if (machine.state == CSVState::INVALID) { - return; - } - if (machine.cur_rows < STANDARD_VECTOR_SIZE && machine.state == CSVState::DELIMITER) { - sniffed_column_counts[machine.cur_rows] = ++machine.column_count; - } - 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) { - suggested_newline = NewLineIdentifier::MIX; - } else { - suggested_newline = NewLineIdentifier::CARRY_ON; - } - } else { - suggested_newline = NewLineIdentifier::SINGLE; - } - if (machine.options.dialect_options.new_line == NewLineIdentifier::NOT_SET) { - machine.dialect_options.new_line = suggested_newline; - } else { - if (machine.options.dialect_options.new_line != suggested_newline) { - // Invalidate this whole detection - machine.cur_rows = 0; - } - } - sniffed_column_counts.erase(sniffed_column_counts.begin() + machine.cur_rows, sniffed_column_counts.end()); - } -}; - bool IsQuoteDefault(char quote) { if (quote == '\"' || quote == '\'' || quote == '\0') { return true; @@ -133,12 +55,18 @@ void CSVSniffer::GenerateCandidateDetectionSearchSpace(vector &delim_candi } } -void CSVSniffer::GenerateStateMachineSearchSpace(vector> &csv_state_machines, +void CSVSniffer::GenerateStateMachineSearchSpace(vector> &column_count_scanners, const vector &delimiter_candidates, const vector "erule_candidates, const unordered_map> "e_candidates_map, const unordered_map> &escape_candidates_map) { // Generate state machines for all option combinations + NewLineIdentifier new_line_id; + if (options.dialect_options.state_machine_options.new_line.IsSetByUser()) { + new_line_id = options.dialect_options.state_machine_options.new_line.GetValue(); + } else { + new_line_id = DetectNewLineDelimiter(*buffer_manager); + } for (const auto quoterule : quoterule_candidates) { const auto "e_candidates = quote_candidates_map.at((uint8_t)quoterule); for (const auto "e : quote_candidates) { @@ -146,36 +74,37 @@ void CSVSniffer::GenerateStateMachineSearchSpace(vector(options, state_machine_options, - buffer_manager, state_machine_cache)); + CSVStateMachineOptions state_machine_options(delimiter, quote, escape, new_line_id); + auto sniffing_state_machine = + make_uniq(options, state_machine_options, state_machine_cache); + column_count_scanners.emplace_back(make_uniq( + buffer_manager, std::move(sniffing_state_machine), detection_error_handler)); } } } } } -void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machine, idx_t &rows_read, +void CSVSniffer::AnalyzeDialectCandidate(unique_ptr scanner, idx_t &rows_read, idx_t &best_consistent_rows, idx_t &prev_padding_count) { // The sniffed_column_counts variable keeps track of the number of columns found for each row - vector sniffed_column_counts(STANDARD_VECTOR_SIZE); - - state_machine->csv_buffer_iterator.Process(*state_machine, sniffed_column_counts); + auto &sniffed_column_counts = scanner->ParseChunk(); idx_t start_row = options.dialect_options.skip_rows.GetValue(); idx_t consistent_rows = 0; - idx_t num_cols = sniffed_column_counts.empty() ? 0 : sniffed_column_counts[0]; + idx_t num_cols = sniffed_column_counts.result_position == 0 ? 1 : sniffed_column_counts[start_row]; idx_t padding_count = 0; bool allow_padding = options.null_padding; - if (sniffed_column_counts.size() > rows_read) { - rows_read = sniffed_column_counts.size(); + if (sniffed_column_counts.result_position > rows_read) { + rows_read = sniffed_column_counts.result_position; } - if (set_columns.IsCandidateUnacceptable(num_cols, options.null_padding, options.ignore_errors)) { + if (set_columns.IsCandidateUnacceptable(num_cols, options.null_padding, options.ignore_errors, + sniffed_column_counts.last_value_always_empty)) { // Not acceptable return; } - for (idx_t row = 0; row < sniffed_column_counts.size(); row++) { - if (set_columns.IsCandidateUnacceptable(sniffed_column_counts[row], options.null_padding, - options.ignore_errors)) { + for (idx_t row = start_row; row < sniffed_column_counts.result_position; row++) { + if (set_columns.IsCandidateUnacceptable(sniffed_column_counts[row], options.null_padding, options.ignore_errors, + sniffed_column_counts.last_value_always_empty)) { // Not acceptable return; } @@ -187,7 +116,7 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi padding_count = 0; // we use the maximum amount of num_cols that we find num_cols = sniffed_column_counts[row]; - start_row = row + options.dialect_options.skip_rows.GetValue(); + start_row = row; consistent_rows = 1; } else if (num_cols >= sniffed_column_counts[row]) { @@ -209,13 +138,12 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi bool require_less_padding = padding_count < prev_padding_count; // If there was only a single column before, and the new number of columns exceeds that. - bool single_column_before = max_columns_found < 2 && num_cols > max_columns_found; + bool single_column_before = max_columns_found < 2 && num_cols > max_columns_found * candidates.size(); // If the number of rows is consistent with the calculated value after accounting for skipped rows and the // start row. - bool rows_consistent = - start_row + consistent_rows - options.dialect_options.skip_rows.GetValue() == sniffed_column_counts.size(); - + bool rows_consistent = consistent_rows + (start_row - options.dialect_options.skip_rows.GetValue()) == + sniffed_column_counts.result_position - options.dialect_options.skip_rows.GetValue(); // If there are more than one consistent row. bool more_than_one_row = (consistent_rows > 1); @@ -223,7 +151,8 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi bool more_than_one_column = (num_cols > 1); // If the start position is valid. - bool start_good = !candidates.empty() && (start_row <= candidates.front()->start_row); + bool start_good = !candidates.empty() && + (start_row <= candidates.front()->GetStateMachine().dialect_options.skip_rows.GetValue()); // If padding happened but it is not allowed. bool invalid_padding = !allow_padding && padding_count > 0; @@ -241,43 +170,48 @@ void CSVSniffer::AnalyzeDialectCandidate(unique_ptr state_machi // We have a candidate that fits our requirements better return; } + auto &sniffing_state_machine = scanner->GetStateMachine(); + best_consistent_rows = consistent_rows; max_columns_found = num_cols; prev_padding_count = padding_count; - state_machine->start_row = start_row; + sniffing_state_machine.dialect_options.skip_rows = start_row; candidates.clear(); - state_machine->dialect_options.num_cols = num_cols; - candidates.emplace_back(std::move(state_machine)); + sniffing_state_machine.dialect_options.num_cols = num_cols; + candidates.emplace_back(std::move(scanner)); return; } // If there's more than one row and column, the start is good, rows are consistent, // no additional padding is required, and there is no invalid padding, and there is not yet a candidate // with the same quote, we add this state_machine as a suitable candidate. if (more_than_one_row && more_than_one_column && start_good && rows_consistent && !require_more_padding && - !invalid_padding) { + !invalid_padding && num_cols == max_columns_found) { + auto &sniffing_state_machine = scanner->GetStateMachine(); + bool same_quote_is_candidate = false; for (auto &candidate : candidates) { - if (state_machine->dialect_options.state_machine_options.quote == - candidate->dialect_options.state_machine_options.quote) { + if (sniffing_state_machine.dialect_options.state_machine_options.quote == + candidate->GetStateMachine().dialect_options.state_machine_options.quote) { same_quote_is_candidate = true; } } if (!same_quote_is_candidate) { - state_machine->start_row = start_row; - state_machine->dialect_options.num_cols = num_cols; - candidates.emplace_back(std::move(state_machine)); + sniffing_state_machine.dialect_options.skip_rows = start_row; + sniffing_state_machine.dialect_options.num_cols = num_cols; + candidates.emplace_back(std::move(scanner)); } } } -bool CSVSniffer::RefineCandidateNextChunk(CSVStateMachine &candidate) { - vector sniffed_column_counts(STANDARD_VECTOR_SIZE); - candidate.csv_buffer_iterator.Process(candidate, sniffed_column_counts); - for (auto &num_cols : sniffed_column_counts) { +bool CSVSniffer::RefineCandidateNextChunk(ColumnCountScanner &candidate) { + auto &sniffed_column_counts = candidate.ParseChunk(); + for (idx_t i = 0; i < sniffed_column_counts.result_position; i++) { if (set_columns.IsSet()) { - return !set_columns.IsCandidateUnacceptable(num_cols, options.null_padding, options.ignore_errors); + return !set_columns.IsCandidateUnacceptable(sniffed_column_counts[i], options.null_padding, + options.ignore_errors, + sniffed_column_counts.last_value_always_empty); } else { - if (max_columns_found != num_cols && (!options.null_padding && !options.ignore_errors)) { + if (max_columns_found != sniffed_column_counts[i] && (!options.null_padding && !options.ignore_errors)) { return false; } } @@ -292,13 +226,13 @@ void CSVSniffer::RefineCandidates() { // No candidates to refine return; } - if (candidates.size() == 1 || candidates[0]->csv_buffer_iterator.Finished()) { + if (candidates.size() == 1 || candidates[0]->FinishedFile()) { // Only one candidate nothing to refine or all candidates already checked return; } for (auto &cur_candidate : candidates) { for (idx_t i = 1; i <= options.sample_size_chunks; i++) { - bool finished_file = cur_candidate->csv_buffer_iterator.Finished(); + bool finished_file = cur_candidate->FinishedFile(); if (finished_file || i == options.sample_size_chunks) { // we finished the file or our chunk sample successfully: stop auto successful_candidate = std::move(cur_candidate); @@ -306,9 +240,7 @@ void CSVSniffer::RefineCandidates() { candidates.emplace_back(std::move(successful_candidate)); return; } - cur_candidate->cur_rows = 0; - cur_candidate->column_count = 1; - if (!RefineCandidateNextChunk(*cur_candidate)) { + if (!RefineCandidateNextChunk(*cur_candidate) || cur_candidate->GetResult().error) { // This candidate failed, move to the next one break; } @@ -318,6 +250,28 @@ void CSVSniffer::RefineCandidates() { return; } +NewLineIdentifier CSVSniffer::DetectNewLineDelimiter(CSVBufferManager &buffer_manager) { + // Get first buffer + auto buffer = buffer_manager.GetBuffer(0); + auto buffer_ptr = buffer->Ptr(); + bool carriage_return = false; + bool n = false; + for (idx_t i = 0; i < buffer->actual_size; i++) { + if (buffer_ptr[i] == '\r') { + carriage_return = true; + } else if (buffer_ptr[i] == '\n') { + n = true; + break; + } else if (carriage_return) { + break; + } + } + if (carriage_return && n) { + return NewLineIdentifier::CARRY_ON; + } + return NewLineIdentifier::SINGLE; +} + // Dialect Detection consists of five steps: // 1. Generate a search space of all possible dialects // 2. Generate a state machine for each dialect @@ -343,7 +297,7 @@ void CSVSniffer::DetectDialect() { // If padding was necessary (i.e., rows are missing some columns, how many) idx_t prev_padding_count = 0; // Vector of CSV State Machines - vector> csv_state_machines; + vector> csv_state_machines; // Step 1: Generate search space GenerateCandidateDetectionSearchSpace(delim_candidates, quoterule_candidates, quote_candidates_map, @@ -358,11 +312,11 @@ void CSVSniffer::DetectDialect() { } // Step 4: Loop over candidates and find if they can still produce good results for the remaining chunks RefineCandidates(); + // if no dialect candidate was found, we throw an exception if (candidates.empty()) { - throw InvalidInputException( - "Error in file \"%s\": CSV options could not be auto-detected. Consider setting parser options manually.", - options.file_path); + auto error = CSVError::SniffingError(options.file_path); + error_handler->Error(error); } } } // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp index 13e8ddc65..feec4d719 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/header_detection.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/types/cast_helpers.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "utf8proc.hpp" namespace duckdb { @@ -93,18 +94,31 @@ static string NormalizeColumnName(const string &col_name) { return col_name_cleaned; } void CSVSniffer::DetectHeader() { + auto &sniffer_state_machine = best_candidate->GetStateMachine(); + + if (best_header_row.empty()) { + sniffer_state_machine.dialect_options.header = false; + for (idx_t col = 0; col < sniffer_state_machine.dialect_options.num_cols; col++) { + names.push_back(GenerateColumnName(sniffer_state_machine.dialect_options.num_cols, col)); + } + // If the user provided names, we must replace our header with the user provided names + for (idx_t i = 0; i < MinValue(names.size(), sniffer_state_machine.options.name_list.size()); i++) { + names[i] = sniffer_state_machine.options.name_list[i]; + } + return; + } // information for header detection bool first_row_consistent = true; // check if header row is all null and/or consistent with detected column data types bool first_row_nulls = true; // If null-padding is not allowed and there is a mismatch between our header candidate and the number of columns // We can't detect the dialect/type options properly - if (!best_candidate->options.null_padding && + if (!sniffer_state_machine.options.null_padding && best_sql_types_candidates_per_column_idx.size() != best_header_row.size()) { - throw InvalidInputException( - "Error in file \"%s\": CSV options could not be auto-detected. Consider setting parser options manually.", - options.file_path); + auto error = CSVError::SniffingError(options.file_path); + error_handler->Error(error); } + bool all_varchar = true; for (idx_t col = 0; col < best_header_row.size(); col++) { auto dummy_val = best_header_row[col]; if (!dummy_val.IsNull()) { @@ -113,19 +127,29 @@ void CSVSniffer::DetectHeader() { // try cast to sql_type of column const auto &sql_type = best_sql_types_candidates_per_column_idx[col].back(); - if (!TryCastValue(*best_candidate, dummy_val, sql_type)) { - first_row_consistent = false; + if (sql_type != LogicalType::VARCHAR) { + all_varchar = false; + if (!TryCastValue(sniffer_state_machine, dummy_val, sql_type)) { + first_row_consistent = false; + } } } bool has_header; - if (!best_candidate->options.dialect_options.header.IsSetByUser()) { - has_header = !first_row_consistent || first_row_nulls; + if (!sniffer_state_machine.options.dialect_options.header.IsSetByUser()) { + has_header = (!first_row_consistent || first_row_nulls) && !all_varchar; } else { - has_header = best_candidate->options.dialect_options.header.GetValue(); + has_header = sniffer_state_machine.options.dialect_options.header.GetValue(); } // update parser info, and read, generate & set col_names based on previous findings if (has_header) { - best_candidate->dialect_options.header = true; + sniffer_state_machine.dialect_options.header = true; + if (sniffer_state_machine.options.null_padding && + !sniffer_state_machine.options.dialect_options.skip_rows.IsSetByUser()) { + if (sniffer_state_machine.dialect_options.skip_rows.GetValue() > 0) { + sniffer_state_machine.dialect_options.skip_rows = + sniffer_state_machine.dialect_options.skip_rows.GetValue() - 1; + } + } case_insensitive_map_t name_collision_count; // get header names from CSV @@ -135,11 +159,11 @@ void CSVSniffer::DetectHeader() { // generate name if field is empty if (col_name.empty() || val.IsNull()) { - col_name = GenerateColumnName(best_candidate->dialect_options.num_cols, col); + col_name = GenerateColumnName(sniffer_state_machine.dialect_options.num_cols, col); } // normalize names or at least trim whitespace - if (best_candidate->options.normalize_names) { + if (sniffer_state_machine.options.normalize_names) { col_name = NormalizeColumnName(col_name); } else { col_name = TrimWhitespace(col_name); @@ -153,24 +177,24 @@ void CSVSniffer::DetectHeader() { names.push_back(col_name); name_collision_count[col_name] = 0; } - if (best_header_row.size() < best_candidate->dialect_options.num_cols && options.null_padding) { - for (idx_t col = best_header_row.size(); col < best_candidate->dialect_options.num_cols; col++) { - names.push_back(GenerateColumnName(best_candidate->dialect_options.num_cols, col)); + if (best_header_row.size() < sniffer_state_machine.dialect_options.num_cols && options.null_padding) { + for (idx_t col = best_header_row.size(); col < sniffer_state_machine.dialect_options.num_cols; col++) { + names.push_back(GenerateColumnName(sniffer_state_machine.dialect_options.num_cols, col)); } - } else if (best_header_row.size() < best_candidate->dialect_options.num_cols) { + } else if (best_header_row.size() < sniffer_state_machine.dialect_options.num_cols) { throw InternalException("Detected header has number of columns inferior to dialect detection"); } } else { - best_candidate->dialect_options.header = false; - for (idx_t col = 0; col < best_candidate->dialect_options.num_cols; col++) { - names.push_back(GenerateColumnName(best_candidate->dialect_options.num_cols, col)); + sniffer_state_machine.dialect_options.header = false; + for (idx_t col = 0; col < sniffer_state_machine.dialect_options.num_cols; col++) { + names.push_back(GenerateColumnName(sniffer_state_machine.dialect_options.num_cols, col)); } } // If the user provided names, we must replace our header with the user provided names - for (idx_t i = 0; i < MinValue(names.size(), best_candidate->options.name_list.size()); i++) { - names[i] = best_candidate->options.name_list[i]; + for (idx_t i = 0; i < MinValue(names.size(), sniffer_state_machine.options.name_list.size()); i++) { + names[i] = sniffer_state_machine.options.name_list[i]; } } } // namespace duckdb 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 9200c06d8..b18318ac0 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 @@ -1,5 +1,5 @@ #include "duckdb/common/operator/decimal_cast_operators.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" #include "duckdb/common/algorithm.hpp" #include "duckdb/common/string.hpp" @@ -13,13 +13,6 @@ struct TryCastFloatingOperator { } }; -struct TupleSniffing { - idx_t line_number; - idx_t position; - bool set = false; - vector values; -}; - static bool StartsWithNumericDate(string &separator, const string &value) { auto begin = value.c_str(); auto end = begin + value.size(); @@ -125,83 +118,6 @@ void CSVSniffer::SetDateFormat(CSVStateMachine &candidate, const string &format_ candidate.dialect_options.date_format[sql_type].Set(strpformat, false); } -struct SniffValue { - inline static void Initialize(CSVStateMachine &machine) { - machine.state = CSVState::STANDARD; - machine.previous_state = CSVState::STANDARD; - machine.pre_previous_state = CSVState::STANDARD; - machine.cur_rows = 0; - machine.value = ""; - machine.rows_read = 0; - } - - inline static bool Process(CSVStateMachine &machine, vector &sniffed_values, char current_char, - idx_t current_pos) { - - if ((machine.dialect_options.new_line == NewLineIdentifier::SINGLE && - (current_char == '\r' || current_char == '\n')) || - (machine.dialect_options.new_line == NewLineIdentifier::CARRY_ON && current_char == '\n')) { - machine.rows_read++; - } - - if ((machine.previous_state == CSVState::RECORD_SEPARATOR) || - (machine.state != CSVState::RECORD_SEPARATOR && machine.previous_state == CSVState::CARRIAGE_RETURN)) { - sniffed_values[machine.cur_rows].position = machine.line_start_pos; - sniffed_values[machine.cur_rows].set = true; - machine.line_start_pos = current_pos; - } - - machine.Transition(current_char); - - bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; - if (machine.previous_state == CSVState::DELIMITER || (machine.previous_state == CSVState::RECORD_SEPARATOR) || - (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) { - // Started a new value - // Check if it's UTF-8 - machine.VerifyUTF8(); - if (machine.value.empty() || machine.value == machine.options.null_str) { - // We set empty == null value - sniffed_values[machine.cur_rows].values.push_back(Value(LogicalType::VARCHAR)); - } else { - sniffed_values[machine.cur_rows].values.push_back(Value(machine.value)); - } - sniffed_values[machine.cur_rows].line_number = machine.rows_read; - - machine.value = ""; - } - if (machine.state == CSVState::STANDARD || - (machine.state == CSVState::QUOTED && machine.previous_state == CSVState::QUOTED)) { - machine.value += current_char; - } - machine.cur_rows += machine.previous_state == CSVState::RECORD_SEPARATOR; - // It means our carriage return is actually a record separator - machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return; - if (machine.cur_rows >= sniffed_values.size()) { - // We sniffed enough rows - return true; - } - return false; - } - - inline static void Finalize(CSVStateMachine &machine, vector &sniffed_values) { - if (machine.cur_rows < sniffed_values.size() && machine.state == CSVState::DELIMITER) { - // Started a new empty value - sniffed_values[machine.cur_rows].values.push_back(Value(machine.value)); - } - if (machine.cur_rows < sniffed_values.size() && machine.state != CSVState::EMPTY_LINE) { - machine.VerifyUTF8(); - sniffed_values[machine.cur_rows].line_number = machine.rows_read; - if (!sniffed_values[machine.cur_rows].set) { - sniffed_values[machine.cur_rows].position = machine.line_start_pos; - sniffed_values[machine.cur_rows].set = true; - } - - sniffed_values[machine.cur_rows++].values.push_back(Value(machine.value)); - } - sniffed_values.erase(sniffed_values.end() - (sniffed_values.size() - machine.cur_rows), sniffed_values.end()); - } -}; - void CSVSniffer::InitializeDateAndTimeStampDetection(CSVStateMachine &candidate, const string &separator, const LogicalType &sql_type) { auto &format_candidate = format_candidates[sql_type.id()]; @@ -270,103 +186,59 @@ void CSVSniffer::DetectTypes() { idx_t min_varchar_cols = max_columns_found + 1; vector return_types; // check which info candidate leads to minimum amount of non-varchar columns... - for (auto &candidate : candidates) { + for (auto &candidate_cc : candidates) { + auto &sniffing_state_machine = candidate_cc->GetStateMachine(); unordered_map> info_sql_types_candidates; - for (idx_t i = 0; i < candidate->dialect_options.num_cols; i++) { - info_sql_types_candidates[i] = candidate->options.auto_type_candidates; + for (idx_t i = 0; i < max_columns_found; i++) { + info_sql_types_candidates[i] = sniffing_state_machine.options.auto_type_candidates; } - D_ASSERT(candidate->dialect_options.num_cols > 0); + D_ASSERT(max_columns_found > 0); - // Set all return_types to VARCHAR so we can do datatype detection based on VARCHAR values + // Set all return_types to VARCHAR, so we can do datatype detection based on VARCHAR values return_types.clear(); - return_types.assign(candidate->dialect_options.num_cols, LogicalType::VARCHAR); + return_types.assign(max_columns_found, LogicalType::VARCHAR); // Reset candidate for parsing - candidate->Reset(); + auto candidate = candidate_cc->UpgradeToStringValueScanner(); // Parse chunk and read csv with info candidate - vector tuples; - idx_t true_line_start = 0; - idx_t true_pos = 0; - do { - tuples.resize(STANDARD_VECTOR_SIZE); - true_line_start = 0; - true_pos = 0; - candidate->csv_buffer_iterator.Process(*candidate, tuples); - // Potentially Skip empty rows (I find this dirty, but it is what the original code does) - // The true line where parsing starts in reference to the csv file - - // The start point of the tuples - idx_t tuple_true_start = 0; - while (tuple_true_start < tuples.size()) { - if (tuples[tuple_true_start].values.empty() || - (tuples[tuple_true_start].values.size() == 1 && tuples[tuple_true_start].values[0].IsNull())) { - true_line_start = tuples[tuple_true_start].line_number; - true_pos = tuples[tuple_true_start].position; - tuple_true_start++; - } else { - break; - } - } - - // Potentially Skip Notes (I also find this dirty, but it is what the original code does) - while (tuple_true_start < tuples.size()) { - if (tuples[tuple_true_start].values.size() < max_columns_found && !options.null_padding) { - true_line_start = tuples[tuple_true_start].line_number; - true_pos = tuples[tuple_true_start].position; - tuple_true_start++; - } else { - break; - } - } - if (tuple_true_start < tuples.size()) { - true_pos = tuples[tuple_true_start].position; - } - if (tuple_true_start > 0) { - tuples.erase(tuples.begin(), tuples.begin() + tuple_true_start); - } - } while (tuples.empty() && !candidate->csv_buffer_iterator.Finished()); - + auto &tuples = candidate->ParseChunk(); idx_t row_idx = 0; - if (tuples.size() > 1 && + if (tuples.number_of_rows > 1 && (!options.dialect_options.header.IsSetByUser() || (options.dialect_options.header.IsSetByUser() && options.dialect_options.header.GetValue()))) { // This means we have more than one row, hence we can use the first row to detect if we have a header row_idx = 1; } - if (!tuples.empty()) { - best_start_without_header = tuples[0].position - true_pos; - } - // First line where we start our type detection 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]; + for (; row_idx < tuples.number_of_rows; row_idx++) { + for (idx_t col_idx = 0; col_idx < tuples.number_of_columns; col_idx++) { + auto &col_type_candidates = info_sql_types_candidates[col_idx]; // 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 D_ASSERT(!col_type_candidates.empty()); auto cur_top_candidate = col_type_candidates.back(); - auto dummy_val = tuples[row_idx].values[col]; + auto dummy_val = tuples.GetValue(row_idx, col_idx); // try cast from string to sql_type while (col_type_candidates.size() > 1) { - // auto cur_top_candidate = col_type_candidates.back(); const auto &sql_type = col_type_candidates.back(); - // try formatting for date types if the user did not specify one and it starts with numeric values. + // try formatting for date types if the user did not specify one and it starts with numeric + // values. string separator; - // If Value is not Null, Has a numeric date format, and the current investigated candidate is either - // a timestamp or a date + // If Value is not Null, Has a numeric date format, and the current investigated candidate is + // either a timestamp or a date if (!dummy_val.IsNull() && StartsWithNumericDate(separator, StringValue::Get(dummy_val)) && (col_type_candidates.back().id() == LogicalTypeId::TIMESTAMP || col_type_candidates.back().id() == LogicalTypeId::DATE)) { - DetectDateAndTimeStampFormats(*candidate, sql_type, separator, dummy_val); + DetectDateAndTimeStampFormats(candidate->GetStateMachine(), sql_type, separator, dummy_val); } // try cast from string to sql_type - if (TryCastValue(*candidate, dummy_val, sql_type)) { + if (sql_type == LogicalType::VARCHAR) { + // Nothing to convert it to + continue; + } + if (TryCastValue(sniffing_state_machine, dummy_val, sql_type)) { break; } else { if (row_idx != start_idx_detection && cur_top_candidate == LogicalType::BOOLEAN) { @@ -394,26 +266,30 @@ void CSVSniffer::DetectTypes() { } } - // it's good if the dialect creates more non-varchar columns, but only if we sacrifice < 30% of best_num_cols. + // it's good if the dialect creates more non-varchar columns, but only if we sacrifice < 30% of + // best_num_cols. if (varchar_cols < min_varchar_cols && info_sql_types_candidates.size() > (max_columns_found * 0.7)) { + best_header_row.clear(); // we have a new best_options candidate - if (true_line_start > 0) { - // Add empty rows to skip_rows - candidate->dialect_options.skip_rows.Set( - candidate->dialect_options.skip_rows.GetValue() + true_line_start, false); - } best_candidate = std::move(candidate); min_varchar_cols = varchar_cols; best_sql_types_candidates_per_column_idx = info_sql_types_candidates; for (auto &format_candidate : format_candidates) { best_format_candidates[format_candidate.first] = format_candidate.second.format; } - best_header_row = tuples[0].values; - best_start_with_header = tuples[0].position - true_pos; + if (tuples.number_of_rows > 0) { + for (idx_t col_idx = 0; col_idx < tuples.number_of_columns; col_idx++) { + best_header_row.emplace_back(tuples.GetValue(0, col_idx)); + } + } } } + if (!best_candidate) { + auto error = CSVError::SniffingError(options.file_path); + error_handler->Error(error); + } // Assert that it's all good at this point. - D_ASSERT(best_candidate && !best_format_candidates.empty() && !best_header_row.empty()); + D_ASSERT(best_candidate && !best_format_candidates.empty()); } } // namespace duckdb 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 6e059dc69..0695980ad 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 @@ -1,116 +1,26 @@ -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" -#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" -namespace duckdb { -struct Parse { - inline static void Initialize(CSVStateMachine &machine) { - machine.state = CSVState::EMPTY_LINE; - machine.previous_state = CSVState::EMPTY_LINE; - machine.pre_previous_state = CSVState::EMPTY_LINE; - - machine.cur_rows = 0; - machine.column_count = 0; - machine.value = ""; - } - - inline static bool Process(CSVStateMachine &machine, DataChunk &parse_chunk, char current_char, idx_t current_pos) { - - machine.Transition(current_char); - - bool carriage_return = machine.previous_state == CSVState::CARRIAGE_RETURN; - if (machine.previous_state == CSVState::DELIMITER || (machine.previous_state == CSVState::RECORD_SEPARATOR) || - (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) { - // 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; - } - if (machine.column_count >= parse_chunk.ColumnCount()) { - throw InvalidInputException("Error in file \"%s\": CSV options could not be auto-detected. Consider " - "setting parser options manually.", - machine.options.file_path); - } - 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)); - } - machine.value = ""; - } - if (((machine.previous_state == CSVState::RECORD_SEPARATOR && machine.state != CSVState::EMPTY_LINE) || - (machine.state != CSVState::RECORD_SEPARATOR && carriage_return)) && - (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++]; - auto &validity_mask = FlatVector::Validity(v); - validity_mask.SetInvalid(machine.cur_rows); - } - } - if (machine.state == CSVState::STANDARD || - (machine.state == CSVState::QUOTED && machine.previous_state == CSVState::QUOTED)) { - machine.value += current_char; - } - machine.cur_rows += machine.previous_state == CSVState::RECORD_SEPARATOR && machine.column_count > 0; - machine.column_count -= machine.column_count * (machine.previous_state == CSVState::RECORD_SEPARATOR); - - // It means our carriage return is actually a record separator - machine.cur_rows += machine.state != CSVState::RECORD_SEPARATOR && carriage_return && machine.column_count > 0; - machine.column_count -= machine.column_count * (machine.state != CSVState::RECORD_SEPARATOR && carriage_return); - - if (machine.cur_rows >= STANDARD_VECTOR_SIZE) { - // We sniffed enough rows - return true; - } - return false; - } - - inline static void Finalize(CSVStateMachine &machine, DataChunk &parse_chunk) { - if (machine.cur_rows < STANDARD_VECTOR_SIZE && machine.state != CSVState::EMPTY_LINE) { - machine.VerifyUTF8(); - 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); - } - } +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_casting.hpp" - machine.cur_rows++; - } - parse_chunk.SetCardinality(machine.cur_rows); - } -}; +namespace duckdb { bool CSVSniffer::TryCastVector(Vector &parse_chunk_col, idx_t size, const LogicalType &sql_type) { + auto &sniffing_state_machine = best_candidate->GetStateMachine(); // try vector-cast from string to sql_type Vector dummy_result(sql_type); - if (!best_candidate->dialect_options.date_format[LogicalTypeId::DATE].GetValue().Empty() && + if (!sniffing_state_machine.dialect_options.date_format[LogicalTypeId::DATE].GetValue().Empty() && sql_type == LogicalTypeId::DATE) { // use the date format to cast the chunk string error_message; idx_t line_error; - return BaseCSVReader::TryCastDateVector(best_candidate->dialect_options.date_format, parse_chunk_col, - dummy_result, size, error_message, line_error); + return CSVCast::TryCastDateVector(sniffing_state_machine.dialect_options.date_format, parse_chunk_col, + dummy_result, size, error_message, line_error); } - if (!best_candidate->dialect_options.date_format[LogicalTypeId::TIMESTAMP].GetValue().Empty() && + if (!sniffing_state_machine.dialect_options.date_format[LogicalTypeId::TIMESTAMP].GetValue().Empty() && sql_type == LogicalTypeId::TIMESTAMP) { // use the timestamp format to cast the chunk string error_message; - return BaseCSVReader::TryCastTimestampVector(best_candidate->dialect_options.date_format, parse_chunk_col, - dummy_result, size, error_message); + return CSVCast::TryCastTimestampVector(sniffing_state_machine.dialect_options.date_format, parse_chunk_col, + dummy_result, size, error_message); } // target type is not varchar: perform a cast string error_message; @@ -118,16 +28,15 @@ bool CSVSniffer::TryCastVector(Vector &parse_chunk_col, idx_t size, const Logica } void CSVSniffer::RefineTypes() { + auto &sniffing_state_machine = best_candidate->GetStateMachine(); // if data types were provided, exit here if number of columns does not match - detected_types.assign(best_candidate->dialect_options.num_cols, LogicalType::VARCHAR); - if (best_candidate->options.all_varchar) { + detected_types.assign(sniffing_state_machine.dialect_options.num_cols, LogicalType::VARCHAR); + if (sniffing_state_machine.options.all_varchar) { // return all types varchar return; } - DataChunk parse_chunk; - parse_chunk.Initialize(BufferAllocator::Get(buffer_manager->context), detected_types, STANDARD_VECTOR_SIZE); - for (idx_t i = 1; i < best_candidate->options.sample_size_chunks; i++) { - bool finished_file = best_candidate->csv_buffer_iterator.Finished(); + for (idx_t i = 1; i < sniffing_state_machine.options.sample_size_chunks; i++) { + bool finished_file = best_candidate->FinishedFile(); if (finished_file) { // we finished the file: stop // set sql types @@ -135,42 +44,20 @@ void CSVSniffer::RefineTypes() { for (idx_t column_idx = 0; column_idx < best_sql_types_candidates_per_column_idx.size(); column_idx++) { LogicalType d_type = best_sql_types_candidates_per_column_idx[column_idx].back(); if (best_sql_types_candidates_per_column_idx[column_idx].size() == - best_candidate->options.auto_type_candidates.size()) { + sniffing_state_machine.options.auto_type_candidates.size()) { d_type = LogicalType::VARCHAR; } detected_types.push_back(d_type); } return; } - best_candidate->csv_buffer_iterator.Process(*best_candidate, parse_chunk); + auto &parse_chunk = best_candidate->ParseChunk().ToChunk(); + for (idx_t col = 0; col < parse_chunk.ColumnCount(); col++) { vector &col_type_candidates = best_sql_types_candidates_per_column_idx[col]; bool is_bool_type = col_type_candidates.back() == LogicalType::BOOLEAN; while (col_type_candidates.size() > 1) { const auto &sql_type = col_type_candidates.back(); - // narrow down the date formats - if (best_format_candidates.count(sql_type.id())) { - auto &best_type_format_candidates = best_format_candidates[sql_type.id()]; - auto save_format_candidates = best_type_format_candidates; - while (!best_type_format_candidates.empty()) { - if (TryCastVector(parse_chunk.data[col], parse_chunk.size(), sql_type)) { - break; - } - // doesn't work - move to the next one - best_type_format_candidates.pop_back(); - if (!best_type_format_candidates.empty()) { - SetDateFormat(*best_candidate, best_type_format_candidates.back(), sql_type.id()); - } - } - // if none match, then this is not a column of type sql_type, - if (best_type_format_candidates.empty()) { - // so restore the candidates that did work. - best_type_format_candidates.swap(save_format_candidates); - if (!best_type_format_candidates.empty()) { - SetDateFormat(*best_candidate, best_type_format_candidates.back(), sql_type.id()); - } - } - } if (TryCastVector(parse_chunk.data[col], parse_chunk.size(), sql_type)) { break; } else { @@ -194,7 +81,7 @@ void CSVSniffer::RefineTypes() { for (idx_t column_idx = 0; column_idx < best_sql_types_candidates_per_column_idx.size(); column_idx++) { LogicalType d_type = best_sql_types_candidates_per_column_idx[column_idx].back(); if (best_sql_types_candidates_per_column_idx[column_idx].size() == - best_candidate->options.auto_type_candidates.size()) { + best_candidate->GetStateMachine().options.auto_type_candidates.size()) { d_type = LogicalType::VARCHAR; } detected_types.push_back(d_type); diff --git a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp index 2d0685a33..c0f35db49 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/sniffer/type_replacement.cpp @@ -1,39 +1,40 @@ -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" -#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" namespace duckdb { void CSVSniffer::ReplaceTypes() { - if (best_candidate->options.sql_type_list.empty()) { + auto &sniffing_state_machine = best_candidate->GetStateMachine(); + if (sniffing_state_machine.options.sql_type_list.empty()) { return; } // user-defined types were supplied for certain columns // override the types - if (!best_candidate->options.sql_types_per_column.empty()) { + if (!sniffing_state_machine.options.sql_types_per_column.empty()) { // types supplied as name -> value map idx_t found = 0; for (idx_t i = 0; i < names.size(); i++) { - auto it = best_candidate->options.sql_types_per_column.find(names[i]); - if (it != best_candidate->options.sql_types_per_column.end()) { - detected_types[i] = best_candidate->options.sql_type_list[it->second]; + auto it = sniffing_state_machine.options.sql_types_per_column.find(names[i]); + if (it != sniffing_state_machine.options.sql_types_per_column.end()) { + best_sql_types_candidates_per_column_idx[i] = { + sniffing_state_machine.options.sql_type_list[it->second]}; + detected_types[i] = sniffing_state_machine.options.sql_type_list[it->second]; found++; } } - if (!best_candidate->options.file_options.union_by_name && - found < best_candidate->options.sql_types_per_column.size()) { - string error_msg = BufferedCSVReader::ColumnTypesError(options.sql_types_per_column, names); - if (!error_msg.empty()) { - throw BinderException(error_msg); - } + if (!sniffing_state_machine.options.file_options.union_by_name && + found < sniffing_state_machine.options.sql_types_per_column.size()) { + auto error_msg = CSVError::ColumnTypesError(options.sql_types_per_column, names); + auto error_line = LinesPerBoundary(0, 0); + error_handler->Error(error_line, error_msg); } return; } // types supplied as list - if (names.size() < best_candidate->options.sql_type_list.size()) { + if (names.size() < sniffing_state_machine.options.sql_type_list.size()) { throw BinderException("read_csv: %d types were provided, but CSV file only has %d columns", - best_candidate->options.sql_type_list.size(), names.size()); + sniffing_state_machine.options.sql_type_list.size(), names.size()); } - for (idx_t i = 0; i < best_candidate->options.sql_type_list.size(); i++) { - detected_types[i] = best_candidate->options.sql_type_list[i]; + for (idx_t i = 0; i < sniffing_state_machine.options.sql_type_list.size(); i++) { + detected_types[i] = sniffing_state_machine.options.sql_type_list[i]; } } } // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine.cpp b/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine.cpp new file mode 100644 index 000000000..665c5b392 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine.cpp @@ -0,0 +1,22 @@ +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "utf8proc_wrapper.hpp" +#include "duckdb/main/error_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp" + +namespace duckdb { + +CSVStateMachine::CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options_p, + CSVStateMachineCache &csv_state_machine_cache) + : transition_array(csv_state_machine_cache.Get(state_machine_options_p)), + state_machine_options(state_machine_options_p), options(options_p) { + dialect_options.state_machine_options = state_machine_options; +} + +CSVStateMachine::CSVStateMachine(const StateMachine &transition_array_p, const CSVReaderOptions &options_p) + : transition_array(transition_array_p), state_machine_options(options_p.dialect_options.state_machine_options), + options(options_p), dialect_options(options.dialect_options) { + dialect_options.state_machine_options = state_machine_options; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine_cache.cpp b/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine_cache.cpp new file mode 100644 index 000000000..6e2d11f57 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/state_machine/csv_state_machine_cache.cpp @@ -0,0 +1,201 @@ +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp" + +namespace duckdb { + +const string CSVStateMachineCache::STATE_KEY = "CSV_STATE_MACHINE_CACHE"; + +void InitializeTransitionArray(StateMachine &transition_array, const CSVState cur_state, const CSVState state) { + for (uint32_t i = 0; i < StateMachine::NUM_TRANSITIONS; i++) { + transition_array[i][static_cast(cur_state)] = state; + } +} +void CSVStateMachineCache::Insert(const CSVStateMachineOptions &state_machine_options) { + D_ASSERT(state_machine_cache.find(state_machine_options) == state_machine_cache.end()); + // Initialize transition array with default values to the Standard option + auto &transition_array = state_machine_cache[state_machine_options]; + + for (uint32_t i = 0; i < StateMachine::NUM_STATES; i++) { + CSVState cur_state = CSVState(i); + switch (cur_state) { + case CSVState::QUOTED: + case CSVState::QUOTED_NEW_LINE: + InitializeTransitionArray(transition_array, cur_state, CSVState::QUOTED); + break; + case CSVState::UNQUOTED: + case CSVState::INVALID: + case CSVState::ESCAPE: + InitializeTransitionArray(transition_array, cur_state, CSVState::INVALID); + break; + default: + InitializeTransitionArray(transition_array, cur_state, CSVState::STANDARD); + break; + } + } + + uint8_t delimiter = static_cast(state_machine_options.delimiter.GetValue()); + uint8_t quote = static_cast(state_machine_options.quote.GetValue()); + uint8_t escape = static_cast(state_machine_options.escape.GetValue()); + + auto new_line_id = state_machine_options.new_line.GetValue(); + + // Now set values depending on configuration + // 1) Standard State + transition_array[delimiter][static_cast(static_cast(CSVState::STANDARD))] = CSVState::DELIMITER; + transition_array[static_cast('\n')][static_cast(CSVState::STANDARD)] = CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::STANDARD)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::STANDARD)] = + CSVState::RECORD_SEPARATOR; + } + // 2) Field Separator State + transition_array[delimiter][static_cast(CSVState::DELIMITER)] = CSVState::DELIMITER; + transition_array[static_cast('\n')][static_cast(CSVState::DELIMITER)] = + CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::DELIMITER)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::DELIMITER)] = + CSVState::RECORD_SEPARATOR; + } + transition_array[quote][static_cast(CSVState::DELIMITER)] = CSVState::QUOTED; + if (delimiter != ' ') { + transition_array[' '][static_cast(CSVState::DELIMITER)] = CSVState::EMPTY_SPACE; + } + + // 3) Record Separator State + transition_array[delimiter][static_cast(CSVState::RECORD_SEPARATOR)] = CSVState::DELIMITER; + transition_array[static_cast('\n')][static_cast(CSVState::RECORD_SEPARATOR)] = + CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::RECORD_SEPARATOR)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::RECORD_SEPARATOR)] = + CSVState::RECORD_SEPARATOR; + } + transition_array[quote][static_cast(CSVState::RECORD_SEPARATOR)] = CSVState::QUOTED; + if (delimiter != ' ') { + transition_array[' '][static_cast(CSVState::RECORD_SEPARATOR)] = CSVState::EMPTY_SPACE; + } + + // 4) Carriage Return State + transition_array[static_cast('\n')][static_cast(CSVState::CARRIAGE_RETURN)] = + CSVState::RECORD_SEPARATOR; + transition_array[static_cast('\r')][static_cast(CSVState::CARRIAGE_RETURN)] = + CSVState::CARRIAGE_RETURN; + transition_array[quote][static_cast(CSVState::CARRIAGE_RETURN)] = CSVState::QUOTED; + if (delimiter != ' ') { + transition_array[' '][static_cast(CSVState::CARRIAGE_RETURN)] = CSVState::EMPTY_SPACE; + } + + // 5) Quoted State + transition_array[quote][static_cast(CSVState::QUOTED)] = CSVState::UNQUOTED; + transition_array['\n'][static_cast(CSVState::QUOTED)] = CSVState::QUOTED_NEW_LINE; + transition_array['\r'][static_cast(CSVState::QUOTED)] = CSVState::QUOTED_NEW_LINE; + + if (state_machine_options.quote != state_machine_options.escape) { + transition_array[escape][static_cast(CSVState::QUOTED)] = CSVState::ESCAPE; + } + // 6) Unquoted State + transition_array[static_cast('\n')][static_cast(CSVState::UNQUOTED)] = CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::UNQUOTED)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::UNQUOTED)] = + CSVState::RECORD_SEPARATOR; + } + transition_array[delimiter][static_cast(CSVState::UNQUOTED)] = CSVState::DELIMITER; + if (state_machine_options.quote == state_machine_options.escape) { + transition_array[escape][static_cast(CSVState::UNQUOTED)] = CSVState::QUOTED; + } + // 7) Escaped State + transition_array[quote][static_cast(CSVState::ESCAPE)] = CSVState::QUOTED; + transition_array[escape][static_cast(CSVState::ESCAPE)] = CSVState::QUOTED; + + // 8) Not Set + transition_array[delimiter][static_cast(static_cast(CSVState::NOT_SET))] = CSVState::DELIMITER; + transition_array[static_cast('\n')][static_cast(CSVState::NOT_SET)] = CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::NOT_SET)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::NOT_SET)] = + CSVState::RECORD_SEPARATOR; + } + transition_array[static_cast(quote)][static_cast(CSVState::NOT_SET)] = CSVState::QUOTED; + if (delimiter != ' ') { + transition_array[' '][static_cast(CSVState::NOT_SET)] = CSVState::EMPTY_SPACE; + } + // 9) Quoted NewLine + transition_array[quote][static_cast(CSVState::QUOTED_NEW_LINE)] = CSVState::UNQUOTED; + if (state_machine_options.quote != state_machine_options.escape) { + transition_array[escape][static_cast(CSVState::QUOTED_NEW_LINE)] = CSVState::ESCAPE; + } + + // 10) Empty Value State + transition_array[delimiter][static_cast(static_cast(CSVState::EMPTY_SPACE))] = + CSVState::DELIMITER; + transition_array[static_cast('\n')][static_cast(CSVState::EMPTY_SPACE)] = + CSVState::RECORD_SEPARATOR; + if (new_line_id == NewLineIdentifier::CARRY_ON) { + transition_array[static_cast('\r')][static_cast(CSVState::EMPTY_SPACE)] = + CSVState::CARRIAGE_RETURN; + } else { + transition_array[static_cast('\r')][static_cast(CSVState::EMPTY_SPACE)] = + CSVState::RECORD_SEPARATOR; + } + transition_array[quote][static_cast(CSVState::EMPTY_SPACE)] = CSVState::QUOTED; + // Initialize characters we can skip during processing, for Standard and Quoted states + for (idx_t i = 0; i < StateMachine::NUM_TRANSITIONS; i++) { + transition_array.skip_standard[i] = true; + transition_array.skip_quoted[i] = true; + } + // For standard states we only care for delimiters \r and \n + transition_array.skip_standard[delimiter] = false; + transition_array.skip_standard[static_cast('\n')] = false; + transition_array.skip_standard[static_cast('\r')] = false; + + // For quoted we only care about quote, escape and for delimiters \r and \n + transition_array.skip_quoted[quote] = false; + transition_array.skip_quoted[escape] = false; + transition_array.skip_quoted[static_cast('\n')] = false; + transition_array.skip_quoted[static_cast('\r')] = false; +} + +CSVStateMachineCache::CSVStateMachineCache() { + for (auto quoterule : default_quote_rule) { + const auto "e_candidates = default_quote[static_cast(quoterule)]; + for (const auto "e : quote_candidates) { + for (const auto &delimiter : default_delimiter) { + const auto &escape_candidates = default_escape[static_cast(quoterule)]; + for (const auto &escape : escape_candidates) { + Insert({delimiter, quote, escape, NewLineIdentifier::SINGLE}); + Insert({delimiter, quote, escape, NewLineIdentifier::CARRY_ON}); + } + } + } + } +} + +const StateMachine &CSVStateMachineCache::Get(const CSVStateMachineOptions &state_machine_options) { + // Custom State Machine, we need to create it and cache it first + lock_guard parallel_lock(main_mutex); + if (state_machine_cache.find(state_machine_options) == state_machine_cache.end()) { + Insert(state_machine_options); + } + const auto &transition_array = state_machine_cache[state_machine_options]; + return transition_array; +} + +CSVStateMachineCache &CSVStateMachineCache::Get(ClientContext &context) { + + auto &cache = ObjectCache::GetObjectCache(context); + return *cache.GetOrCreate(CSVStateMachineCache::STATE_KEY); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/table_function/csv_file_scanner.cpp b/src/duckdb/src/execution/operator/csv_scanner/table_function/csv_file_scanner.cpp new file mode 100644 index 000000000..cca2c5230 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/table_function/csv_file_scanner.cpp @@ -0,0 +1,226 @@ +#include "duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp" +#include "duckdb/function/table/read_csv.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" + +namespace duckdb { + +CSVFileScan::CSVFileScan(ClientContext &context, shared_ptr buffer_manager_p, + shared_ptr state_machine_p, const CSVReaderOptions &options_p, + const ReadCSVData &bind_data, const vector &column_ids, + vector &file_schema) + : file_path(options_p.file_path), file_idx(0), buffer_manager(std::move(buffer_manager_p)), + state_machine(std::move(state_machine_p)), file_size(buffer_manager->file_handle->FileSize()), + error_handler(make_shared(options_p.ignore_errors)), + on_disk_file(buffer_manager->file_handle->OnDiskFile()), options(options_p) { + if (bind_data.initial_reader.get()) { + auto &union_reader = *bind_data.initial_reader; + names = union_reader.GetNames(); + options = union_reader.options; + types = union_reader.GetTypes(); + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, bind_data.return_types, + bind_data.return_names, column_ids, nullptr, file_path, context); + InitializeFileNamesTypes(); + return; + } else if (!bind_data.column_info.empty()) { + // Serialized Union By name + names = bind_data.column_info[0].names; + types = bind_data.column_info[0].types; + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, bind_data.return_types, + bind_data.return_names, column_ids, nullptr, file_path, context); + InitializeFileNamesTypes(); + return; + } + names = bind_data.return_names; + types = bind_data.return_types; + file_schema = bind_data.return_types; + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, bind_data.return_types, + bind_data.return_names, column_ids, nullptr, file_path, context); + + InitializeFileNamesTypes(); +} + +CSVFileScan::CSVFileScan(ClientContext &context, const string &file_path_p, const CSVReaderOptions &options_p, + const idx_t file_idx_p, const ReadCSVData &bind_data, const vector &column_ids, + const vector &file_schema) + : file_path(file_path_p), file_idx(file_idx_p), + error_handler(make_shared(options_p.ignore_errors)), options(options_p) { + if (file_idx < bind_data.union_readers.size()) { + // we are doing UNION BY NAME - fetch the options from the union reader for this file + optional_ptr union_reader_ptr; + if (file_idx == 0) { + union_reader_ptr = bind_data.initial_reader.get(); + } else { + union_reader_ptr = bind_data.union_readers[file_idx].get(); + } + if (union_reader_ptr) { + auto &union_reader = *union_reader_ptr; + // Initialize Buffer Manager + buffer_manager = union_reader.buffer_manager; + // Initialize On Disk and Size of file + on_disk_file = union_reader.on_disk_file; + file_size = union_reader.file_size; + names = union_reader.GetNames(); + options = union_reader.options; + types = union_reader.GetTypes(); + state_machine = union_reader.state_machine; + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, + bind_data.return_types, bind_data.return_names, column_ids, nullptr, + file_path, context); + + InitializeFileNamesTypes(); + return; + } + } + + // Initialize Buffer Manager + buffer_manager = make_shared(context, options, file_path, file_idx); + // Initialize On Disk and Size of file + on_disk_file = buffer_manager->file_handle->OnDiskFile(); + file_size = buffer_manager->file_handle->FileSize(); + // Initialize State Machine + auto &state_machine_cache = CSVStateMachineCache::Get(context); + + if (file_idx < bind_data.column_info.size()) { + // Serialized Union By name + names = bind_data.column_info[file_idx].names; + types = bind_data.column_info[file_idx].types; + options.dialect_options.num_cols = names.size(); + state_machine = make_shared( + state_machine_cache.Get(options.dialect_options.state_machine_options), options); + + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, bind_data.return_types, + bind_data.return_names, column_ids, nullptr, file_path, context); + InitializeFileNamesTypes(); + return; + } + // Sniff it (We only really care about dialect detection, if types or number of columns are different this will + // error out during scanning) + if (options.auto_detect && file_idx > 0) { + CSVSniffer sniffer(options, buffer_manager, state_machine_cache); + auto result = sniffer.SniffCSV(); + if (!file_schema.empty()) { + if (!options.file_options.filename && !options.file_options.hive_partitioning && + file_schema.size() != result.return_types.size()) { + throw InvalidInputException("Mismatch between the schema of different files"); + } + } + } + if (options.dialect_options.num_cols == 0) { + // We need to define the number of columns, if the sniffer is not running this must be in the sql_type_list + options.dialect_options.num_cols = options.sql_type_list.size(); + } + + if (options.dialect_options.state_machine_options.new_line == NewLineIdentifier::NOT_SET) { + options.dialect_options.state_machine_options.new_line = CSVSniffer::DetectNewLineDelimiter(*buffer_manager); + } + + names = bind_data.return_names; + types = bind_data.return_types; + state_machine = + make_shared(state_machine_cache.Get(options.dialect_options.state_machine_options), options); + + MultiFileReader::InitializeReader(*this, options.file_options, bind_data.reader_bind, bind_data.return_types, + bind_data.return_names, column_ids, nullptr, file_path, context); + InitializeFileNamesTypes(); +} + +CSVFileScan::CSVFileScan(ClientContext &context, const string &file_name, CSVReaderOptions &options_p) + : file_path(file_name), file_idx(0), error_handler(make_shared(options_p.ignore_errors)), + options(options_p) { + buffer_manager = make_shared(context, options, file_path, file_idx); + // Initialize On Disk and Size of file + on_disk_file = buffer_manager->file_handle->OnDiskFile(); + file_size = buffer_manager->file_handle->FileSize(); + // Sniff it (We only really care about dialect detection, if types or number of columns are different this will + // error out during scanning) + auto &state_machine_cache = CSVStateMachineCache::Get(context); + if (options.auto_detect && options.dialect_options.num_cols == 0) { + CSVSniffer sniffer(options, buffer_manager, state_machine_cache); + auto sniffer_result = sniffer.SniffCSV(); + if (names.empty()) { + names = sniffer_result.names; + types = sniffer_result.return_types; + } + } + if (options.dialect_options.num_cols == 0) { + // We need to define the number of columns, if the sniffer is not running this must be in the sql_type_list + options.dialect_options.num_cols = options.sql_type_list.size(); + } + // Initialize State Machine + state_machine = + make_shared(state_machine_cache.Get(options.dialect_options.state_machine_options), options); +} + +void CSVFileScan::InitializeFileNamesTypes() { + if (options.null_padding) { + // If we are null padding we do not yet support projection pushdown + file_types = types; + return; + } + if (reader_data.empty_columns && reader_data.column_ids.empty()) { + // This means that the columns from this file are irrelevant. + // just read the first column + file_types.emplace_back(LogicalType::VARCHAR); + projected_columns.insert(0); + projection_ids.emplace_back(0, 0); + return; + } + + for (idx_t i = 0; i < reader_data.column_ids.size(); i++) { + idx_t result_idx = reader_data.column_ids[i]; + file_types.emplace_back(types[result_idx]); + projected_columns.insert(result_idx); + projection_ids.emplace_back(result_idx, i); + } + + if (!projected_columns.empty()) { + // We might have to add recovery rejects column ids + for (idx_t i = 0; i < options.rejects_recovery_column_ids.size(); i++) { + idx_t col_id = options.rejects_recovery_column_ids[i]; + if (projected_columns.find(col_id) == projected_columns.end()) { + // We have to insert this column in our projection + projected_columns.insert(col_id); + file_types.emplace_back(LogicalType::VARCHAR); + projected_columns.insert(col_id); + projection_ids.emplace_back(col_id, col_id); + } + } + } + + if (reader_data.column_ids.empty()) { + file_types = types; + } + + // We need to be sure that our types are also following the cast_map + for (idx_t i = 0; i < reader_data.column_ids.size(); i++) { + if (reader_data.cast_map.find(reader_data.column_ids[i]) != reader_data.cast_map.end()) { + file_types[i] = reader_data.cast_map[reader_data.column_ids[i]]; + } + } + + // We sort the types on the order of the parsed chunk + std::sort(projection_ids.begin(), projection_ids.end()); + vector sorted_types; + for (idx_t i = 0; i < projection_ids.size(); ++i) { + sorted_types.push_back(file_types[projection_ids[i].second]); + } + file_types = sorted_types; +} + +const string &CSVFileScan::GetFileName() { + return file_path; +} +const vector &CSVFileScan::GetNames() { + return names; +} +const vector &CSVFileScan::GetTypes() { + return types; +} + +void CSVFileScan::InitializeProjection() { + for (idx_t i = 0; i < options.dialect_options.num_cols; i++) { + reader_data.column_ids.push_back(i); + reader_data.column_mapping.push_back(i); + } +} +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/table_function/global_csv_state.cpp b/src/duckdb/src/execution/operator/csv_scanner/table_function/global_csv_state.cpp new file mode 100644 index 000000000..b262f592e --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/table_function/global_csv_state.cpp @@ -0,0 +1,197 @@ +#include "duckdb/execution/operator/csv_scanner/global_csv_state.hpp" +#include "duckdb/main/client_data.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" +#include "duckdb/main/appender.hpp" + +namespace duckdb { + +CSVGlobalState::CSVGlobalState(ClientContext &context_p, const shared_ptr &buffer_manager, + const CSVReaderOptions &options, idx_t system_threads_p, const vector &files, + vector column_ids_p, const ReadCSVData &bind_data_p) + : context(context_p), system_threads(system_threads_p), column_ids(std::move(column_ids_p)), + sniffer_mismatch_error(options.sniffer_user_mismatch_error), bind_data(bind_data_p) { + + if (buffer_manager && buffer_manager->GetFilePath() == files[0]) { + auto state_machine = make_shared( + CSVStateMachineCache::Get(context).Get(options.dialect_options.state_machine_options), options); + // If we already have a buffer manager, we don't need to reconstruct it to the first file + file_scans.emplace_back(make_uniq(context, buffer_manager, state_machine, options, bind_data, + column_ids, file_schema)); + } else { + // If not we need to construct it for the first file + file_scans.emplace_back( + make_uniq(context, files[0], options, 0, bind_data, column_ids, file_schema)); + }; + //! There are situations where we only support single threaded scanning + bool many_csv_files = files.size() > 1 && files.size() > system_threads * 2; + single_threaded = many_csv_files || !options.parallel; + last_file_idx = 0; + scanner_idx = 0; + running_threads = MaxThreads(); + if (single_threaded) { + current_boundary = CSVIterator(); + } else { + auto buffer_size = file_scans.back()->buffer_manager->GetBuffer(0)->actual_size; + current_boundary = CSVIterator(0, 0, 0, 0, buffer_size); + } +} + +double CSVGlobalState::GetProgress(const ReadCSVData &bind_data_p) const { + + idx_t total_files = bind_data.files.size(); + // get the progress WITHIN the current file + double progress; + if (file_scans.back()->file_size == 0) { + progress = 1.0; + } else { + // for compressed files, readed bytes may greater than files size. + progress = std::min(1.0, double(file_scans.back()->bytes_read) / double(file_scans.back()->file_size)); + } + // now get the total percentage of files read + double percentage = double(current_boundary.GetFileIdx()) / total_files; + percentage += (double(1) / double(total_files)) * progress; + return percentage * 100; +} + +unique_ptr CSVGlobalState::Next() { + if (single_threaded) { + idx_t cur_idx = last_file_idx++; + if (cur_idx >= bind_data.files.size()) { + return nullptr; + } + shared_ptr current_file; + if (cur_idx == 0) { + current_file = file_scans.back(); + } else { + current_file = make_shared(context, bind_data.files[cur_idx], bind_data.options, cur_idx, + bind_data, column_ids, file_schema); + } + auto csv_scanner = + make_uniq(scanner_idx++, current_file->buffer_manager, current_file->state_machine, + current_file->error_handler, current_file, current_boundary); + return csv_scanner; + } + lock_guard parallel_lock(main_mutex); + if (finished) { + return nullptr; + } + + // We first create the scanner for the current boundary + auto ¤t_file = *file_scans.back(); + auto csv_scanner = + make_uniq(scanner_idx++, current_file.buffer_manager, current_file.state_machine, + current_file.error_handler, file_scans.back(), current_boundary); + // We then produce the next boundary + if (!current_boundary.Next(*current_file.buffer_manager)) { + // This means we are done scanning the current file + auto current_file_idx = current_file.file_idx + 1; + if (current_file_idx < bind_data.files.size()) { + file_scans.back()->buffer_manager.reset(); + // If we have a next file we have to construct the file scan for that + file_scans.emplace_back(make_shared(context, bind_data.files[current_file_idx], + bind_data.options, current_file_idx, bind_data, column_ids, + file_schema)); + // And re-start the boundary-iterator + auto buffer_size = file_scans.back()->buffer_manager->GetBuffer(0)->actual_size; + current_boundary = CSVIterator(current_file_idx, 0, 0, 0, buffer_size); + } else { + // If not we are done with this CSV Scanning + finished = true; + } + } + // We initialize the scan + return csv_scanner; +} + +idx_t CSVGlobalState::MaxThreads() const { + // We initialize max one thread per our set bytes per thread limit + if (single_threaded) { + return system_threads; + } + idx_t total_threads = file_scans.back()->file_size / CSVIterator::BYTES_PER_THREAD + 1; + + if (total_threads < system_threads) { + return total_threads; + } + return system_threads; +} + +void CSVGlobalState::DecrementThread() { + lock_guard parallel_lock(main_mutex); + D_ASSERT(running_threads > 0); + running_threads--; + if (running_threads == 0) { + FillRejectsTable(); + if (context.client_data->debug_set_max_line_length) { + context.client_data->debug_max_line_length = file_scans[0]->error_handler->GetMaxLineLength(); + } + } +} + +void CSVGlobalState::FillRejectsTable() { + auto &options = bind_data.options; + + if (!options.rejects_table_name.empty()) { + auto limit = options.rejects_limit; + + auto rejects = CSVRejectsTable::GetOrCreate(context, options.rejects_table_name); + lock_guard lock(rejects->write_lock); + auto &table = rejects->GetTable(context); + InternalAppender appender(context, table); + + for (auto &file : file_scans) { + auto file_name = file->file_path; + auto &errors = file->error_handler->errors; + for (auto &error_info : errors) { + if (error_info.second.type != CSVErrorType::CAST_ERROR) { + // For now we only will use it for casting errors + continue; + } + // short circuit if we already have too many rejects + if (limit == 0 || rejects->count < limit) { + if (limit != 0 && rejects->count >= limit) { + break; + } + rejects->count++; + auto error = &error_info.second; + auto row_line = file->error_handler->GetLine(error_info.first); + auto col_idx = error->column_idx; + auto col_name = bind_data.return_names[col_idx]; + // Add the row to the rejects table + appender.BeginRow(); + appender.Append(string_t(file_name)); + appender.Append(row_line); + appender.Append(col_idx); + appender.Append(string_t("\"" + col_name + "\"")); + appender.Append(error->row[col_idx]); + + if (!options.rejects_recovery_columns.empty()) { + child_list_t recovery_key; + for (auto &key_idx : options.rejects_recovery_column_ids) { + // Figure out if the recovery key is valid. + // If not, error out for real. + auto &value = error->row[key_idx]; + if (value.IsNull()) { + throw InvalidInputException("%s at line %llu in column %s. Parser options:\n%s ", + "Could not parse recovery column", row_line, col_name, + options.ToString()); + } + recovery_key.emplace_back(bind_data.return_names[key_idx], value); + } + appender.Append(Value::STRUCT(recovery_key)); + } + auto row_error_msg = + StringUtil::Format("Could not convert string '%s' to '%s'", error->row[col_idx].ToString(), + file->types[col_idx].ToString()); + appender.Append(string_t(row_error_msg)); + appender.EndRow(); + } + appender.Close(); + } + } + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/util/csv_error.cpp b/src/duckdb/src/execution/operator/csv_scanner/util/csv_error.cpp new file mode 100644 index 000000000..b2c737c99 --- /dev/null +++ b/src/duckdb/src/execution/operator/csv_scanner/util/csv_error.cpp @@ -0,0 +1,185 @@ +#include "duckdb/execution/operator/csv_scanner/csv_error.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" +#include + +namespace duckdb { + +LinesPerBoundary::LinesPerBoundary() { +} +LinesPerBoundary::LinesPerBoundary(idx_t boundary_idx_p, idx_t lines_in_batch_p) + : boundary_idx(boundary_idx_p), lines_in_batch(lines_in_batch_p) { +} + +CSVErrorHandler::CSVErrorHandler(bool ignore_errors_p) : ignore_errors(ignore_errors_p) { +} + +void CSVErrorHandler::Error(CSVError &csv_error) { + LinesPerBoundary mock; + Error(mock, csv_error, true); +} +void CSVErrorHandler::Error(LinesPerBoundary &error_info, CSVError &csv_error, bool force_error) { + if (ignore_errors && !force_error) { + lock_guard parallel_lock(main_mutex); + // We store this error + errors.push_back({error_info, csv_error}); + return; + } + + std::ostringstream error; + if (PrintLineNumber(csv_error)) { + error << "CSV Error on Line: " << GetLine(error_info) << std::endl; + } + { + lock_guard parallel_lock(main_mutex); + got_borked = true; + } + error << csv_error.error_message; + switch (csv_error.type) { + case CSVErrorType::CAST_ERROR: + throw ConversionException(error.str()); + case CSVErrorType::COLUMN_NAME_TYPE_MISMATCH: + throw BinderException(error.str()); + case CSVErrorType::NULLPADDED_QUOTED_NEW_VALUE: + throw ParameterNotAllowedException(error.str()); + default: + throw InvalidInputException(error.str()); + } +} + +void CSVErrorHandler::Insert(idx_t boundary_idx, idx_t rows) { + lock_guard parallel_lock(main_mutex); + if (lines_per_batch_map.find(boundary_idx) == lines_per_batch_map.end()) { + lines_per_batch_map[boundary_idx] = {boundary_idx, rows}; + } else { + lines_per_batch_map[boundary_idx].lines_in_batch += rows; + } +} + +void CSVErrorHandler::NewMaxLineSize(idx_t scan_line_size) { + lock_guard parallel_lock(main_mutex); + max_line_length = std::max(scan_line_size, max_line_length); +} + +CSVError::CSVError(string error_message_p, CSVErrorType type_p) + : error_message(std::move(error_message_p)), type(type_p) { +} + +CSVError::CSVError(string error_message_p, CSVErrorType type_p, idx_t column_idx_p, vector row_p) + : error_message(std::move(error_message_p)), type(type_p), column_idx(column_idx_p), row(std::move(row_p)) { +} + +CSVError CSVError::ColumnTypesError(case_insensitive_map_t sql_types_per_column, const vector &names) { + for (idx_t i = 0; i < names.size(); i++) { + auto it = sql_types_per_column.find(names[i]); + if (it != sql_types_per_column.end()) { + sql_types_per_column.erase(names[i]); + continue; + } + } + if (sql_types_per_column.empty()) { + return CSVError("", CSVErrorType::COLUMN_NAME_TYPE_MISMATCH); + } + string exception = "COLUMN_TYPES error: Columns with names: "; + for (auto &col : sql_types_per_column) { + exception += "\"" + col.first + "\","; + } + exception.pop_back(); + exception += " do not exist in the CSV File"; + return CSVError(exception, CSVErrorType::COLUMN_NAME_TYPE_MISMATCH); +} + +CSVError CSVError::CastError(const CSVReaderOptions &options, string &column_name, string &cast_error, idx_t column_idx, + vector &row) { + std::ostringstream error; + // Which column + error << "Error when converting column \"" << column_name << "\"." << std::endl; + // What was the cast error + error << cast_error << std::endl; + error << std::endl; + // What were the options + error << options.ToString(); + return CSVError(error.str(), CSVErrorType::CAST_ERROR, column_idx, row); +} + +CSVError CSVError::LineSizeError(const CSVReaderOptions &options, idx_t actual_size) { + std::ostringstream error; + error << "Maximum line size of " << options.maximum_line_size << " bytes exceeded. "; + error << "Actual Size:" << actual_size << " bytes." << std::endl; + error << options.ToString(); + return CSVError(error.str(), CSVErrorType::MAXIMUM_LINE_SIZE); +} + +CSVError CSVError::SniffingError(string &file_path) { + std::ostringstream error; + // Which column + error << "Error when sniffing file \"" << file_path << "\"." << std::endl; + error << "CSV options could not be auto-detected. Consider setting parser options manually." << std::endl; + return CSVError(error.str(), CSVErrorType::SNIFFING); +} + +CSVError CSVError::NullPaddingFail(const CSVReaderOptions &options) { + std::ostringstream error; + error << " The parallel scanner does not support null_padding in conjunction with quoted new lines. Please " + "disable the parallel csv reader with parallel=false" + << std::endl; + // What were the options + error << options.ToString(); + return CSVError(error.str(), CSVErrorType::NULLPADDED_QUOTED_NEW_VALUE); +} + +CSVError CSVError::UnterminatedQuotesError(const CSVReaderOptions &options, string_t *vector_ptr, + idx_t vector_line_start, idx_t current_column) { + std::ostringstream error; + error << "Value with unterminated quote found." << std::endl; + error << std::endl; + // What were the options + error << options.ToString(); + return CSVError(error.str(), CSVErrorType::UNTERMINATED_QUOTES); +} + +CSVError CSVError::IncorrectColumnAmountError(const CSVReaderOptions &options, string_t *vector_ptr, + idx_t vector_line_start, idx_t actual_columns) { + std::ostringstream error; + // How many columns were expected and how many were found + error << "Expected Number of Columns: " << options.dialect_options.num_cols << " Found: " << actual_columns + << std::endl; + // What were the options + error << options.ToString(); + return CSVError(error.str(), CSVErrorType::INCORRECT_COLUMN_AMOUNT); +} + +bool CSVErrorHandler::PrintLineNumber(CSVError &error) { + switch (error.type) { + case CSVErrorType::CAST_ERROR: + case CSVErrorType::UNTERMINATED_QUOTES: + case CSVErrorType::INCORRECT_COLUMN_AMOUNT: + case CSVErrorType::MAXIMUM_LINE_SIZE: + case CSVErrorType::NULLPADDED_QUOTED_NEW_VALUE: + return true; + default: + return false; + } +} + +idx_t CSVErrorHandler::GetLine(LinesPerBoundary &error_info) { + idx_t current_line = 1 + error_info.lines_in_batch; // We start from one, since the lines are 1-indexed + for (idx_t boundary_idx = 0; boundary_idx < error_info.boundary_idx; boundary_idx++) { + bool batch_done = false; + while (!batch_done) { + if (boundary_idx == 0) { + // if it's the first boundary, we just return + break; + } + if (lines_per_batch_map.find(boundary_idx) != lines_per_batch_map.end()) { + batch_done = true; + current_line += lines_per_batch_map[boundary_idx].lines_in_batch; + } + if (got_borked) { + return current_line; + } + } + } + return current_line; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp b/src/duckdb/src/execution/operator/csv_scanner/util/csv_reader_options.cpp similarity index 97% rename from src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp rename to src/duckdb/src/execution/operator/csv_scanner/util/csv_reader_options.cpp index 4710da238..d05db0bb8 100644 --- a/src/duckdb/src/execution/operator/csv_scanner/csv_reader_options.cpp +++ b/src/duckdb/src/execution/operator/csv_scanner/util/csv_reader_options.cpp @@ -1,4 +1,4 @@ -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "duckdb/common/bind_helpers.hpp" #include "duckdb/common/vector_size.hpp" #include "duckdb/common/string_util.hpp" @@ -128,14 +128,14 @@ void CSVReaderOptions::SetQuote(const string "e_p) { } NewLineIdentifier CSVReaderOptions::GetNewline() const { - return dialect_options.new_line.GetValue(); + return dialect_options.state_machine_options.new_line.GetValue(); } void CSVReaderOptions::SetNewline(const string &input) { if (input == "\\n" || input == "\\r") { - dialect_options.new_line.Set(NewLineIdentifier::SINGLE); + dialect_options.state_machine_options.new_line.Set(NewLineIdentifier::SINGLE); } else if (input == "\\r\\n") { - dialect_options.new_line.Set(NewLineIdentifier::CARRY_ON); + dialect_options.state_machine_options.new_line.Set(NewLineIdentifier::CARRY_ON); } else { throw InvalidInputException("This is not accepted as a newline: " + input); } @@ -202,10 +202,10 @@ void CSVReaderOptions::SetReadOption(const string &loption, const Value &value, } } else if (loption == "null_padding") { null_padding = ParseBoolean(value, loption); + } else if (loption == "parallel") { + parallel = ParseBoolean(value, loption); } else if (loption == "allow_quoted_nulls") { allow_quoted_nulls = ParseBoolean(value, loption); - } else if (loption == "parallel") { - parallel_mode = ParseBoolean(value, loption) ? ParallelMode::PARALLEL : ParallelMode::SINGLE_THREADED; } else if (loption == "rejects_table") { // skip, handled in SetRejectsOptions auto table_name = ParseString(value, loption); @@ -301,7 +301,7 @@ string CSVReaderOptions::ToString() const { auto &delimiter = dialect_options.state_machine_options.delimiter; auto "e = dialect_options.state_machine_options.quote; auto &escape = dialect_options.state_machine_options.escape; - auto &new_line = dialect_options.new_line; + auto &new_line = dialect_options.state_machine_options.new_line; auto &skip_rows = dialect_options.skip_rows; auto &header = dialect_options.header; @@ -496,7 +496,7 @@ void CSVReaderOptions::ToNamedParameters(named_parameter_map_t &named_params) { if (delimiter.IsSetByUser()) { named_params["delim"] = Value(GetDelimiter()); } - if (dialect_options.new_line.IsSetByUser()) { + if (dialect_options.state_machine_options.new_line.IsSetByUser()) { named_params["newline"] = Value(EnumUtil::ToString(GetNewline())); } if (quote.IsSetByUser()) { @@ -513,6 +513,7 @@ void CSVReaderOptions::ToNamedParameters(named_parameter_map_t &named_params) { named_params["skip"] = Value::BIGINT(GetSkipRows()); } named_params["null_padding"] = Value::BOOLEAN(null_padding); + named_params["parallel"] = Value::BOOLEAN(parallel); if (!date_format.at(LogicalType::DATE).format_specifier.empty()) { named_params["dateformat"] = Value(date_format.at(LogicalType::DATE).format_specifier); } diff --git a/src/duckdb/src/execution/operator/helper/physical_create_secret.cpp b/src/duckdb/src/execution/operator/helper/physical_create_secret.cpp new file mode 100644 index 000000000..3a1dbe8b6 --- /dev/null +++ b/src/duckdb/src/execution/operator/helper/physical_create_secret.cpp @@ -0,0 +1,21 @@ +#include "duckdb/execution/operator/helper/physical_create_secret.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/main/secret/secret_manager.hpp" + +namespace duckdb { + +SourceResultType PhysicalCreateSecret::GetData(ExecutionContext &context, DataChunk &chunk, + OperatorSourceInput &input) const { + auto &client = context.client; + auto &secret_manager = SecretManager::Get(client); + + secret_manager.CreateSecret(client, info); + + chunk.SetValue(0, 0, true); + chunk.SetCardinality(1); + + return SourceResultType::FINISHED; +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/helper/physical_materialized_collector.cpp b/src/duckdb/src/execution/operator/helper/physical_materialized_collector.cpp index 650a8916c..c5f0309ff 100644 --- a/src/duckdb/src/execution/operator/helper/physical_materialized_collector.cpp +++ b/src/duckdb/src/execution/operator/helper/physical_materialized_collector.cpp @@ -1,5 +1,5 @@ #include "duckdb/execution/operator/helper/physical_materialized_collector.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/main/materialized_query_result.hpp" #include "duckdb/main/client_context.hpp" diff --git a/src/duckdb/src/execution/operator/helper/physical_reservoir_sample.cpp b/src/duckdb/src/execution/operator/helper/physical_reservoir_sample.cpp index ec846f1d4..1bc87d25f 100644 --- a/src/duckdb/src/execution/operator/helper/physical_reservoir_sample.cpp +++ b/src/duckdb/src/execution/operator/helper/physical_reservoir_sample.cpp @@ -6,6 +6,7 @@ namespace duckdb { //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// + class SampleGlobalSinkState : public GlobalSinkState { public: explicit SampleGlobalSinkState(Allocator &allocator, SampleOptions &options) { @@ -24,7 +25,8 @@ class SampleGlobalSinkState : public GlobalSinkState { } } - //! The lock for updating the global aggregate state + //! The lock for updating the global aggoregate state + //! Also used to update the global sample when percentages are used mutex lock; //! The reservoir sample unique_ptr sample; @@ -36,24 +38,47 @@ unique_ptr PhysicalReservoirSample::GetGlobalSinkState(ClientCo SinkResultType PhysicalReservoirSample::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &gstate = input.global_state.Cast(); - if (!gstate.sample) { - return SinkResultType::FINISHED; + auto &global_state = input.global_state.Cast(); + // Percentage only has a global sample. + lock_guard glock(global_state.lock); + if (!global_state.sample) { + // always gather full thread percentage + auto &allocator = Allocator::Get(context.client); + if (options->is_percentage) { + double percentage = options->sample_size.GetValue(); + if (percentage == 0) { + return SinkResultType::FINISHED; + } + global_state.sample = make_uniq(allocator, percentage, options->seed); + } else { + idx_t num_samples = options->sample_size.GetValue(); + if (num_samples == 0) { + return SinkResultType::FINISHED; + } + global_state.sample = make_uniq(allocator, num_samples, options->seed); + } } - // we implement reservoir sampling without replacement and exponential jumps here - // the algorithm is adopted from the paper Weighted random sampling with a reservoir by Pavlos S. Efraimidis et al. - // note that the original algorithm is about weighted sampling; this is a simplified approach for uniform sampling - lock_guard glock(gstate.lock); - gstate.sample->AddToReservoir(chunk); + global_state.sample->AddToReservoir(chunk); return SinkResultType::NEED_MORE_INPUT; } +SinkCombineResultType PhysicalReservoirSample::Combine(ExecutionContext &context, + OperatorSinkCombineInput &input) const { + return SinkCombineResultType::FINISHED; +} + +SinkFinalizeType PhysicalReservoirSample::Finalize(Pipeline &pipeline, Event &event, ClientContext &context, + OperatorSinkFinalizeInput &input) const { + return SinkFinalizeType::READY; +} + //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// SourceResultType PhysicalReservoirSample::GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const { auto &sink = this->sink_state->Cast(); + lock_guard glock(sink.lock); if (!sink.sample) { return SourceResultType::FINISHED; } diff --git a/src/duckdb/src/execution/operator/helper/physical_transaction.cpp b/src/duckdb/src/execution/operator/helper/physical_transaction.cpp index cca98d859..08ad8c599 100644 --- a/src/duckdb/src/execution/operator/helper/physical_transaction.cpp +++ b/src/duckdb/src/execution/operator/helper/physical_transaction.cpp @@ -1,6 +1,7 @@ #include "duckdb/execution/operator/helper/physical_transaction.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/valid_checker.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/execution/operator/helper/physical_vacuum.cpp b/src/duckdb/src/execution/operator/helper/physical_vacuum.cpp index 7ab179abe..10fd47790 100644 --- a/src/duckdb/src/execution/operator/helper/physical_vacuum.cpp +++ b/src/duckdb/src/execution/operator/helper/physical_vacuum.cpp @@ -15,8 +15,13 @@ PhysicalVacuum::PhysicalVacuum(unique_ptr info_p, idx_t estimated_ca class VacuumLocalSinkState : public LocalSinkState { public: explicit VacuumLocalSinkState(VacuumInfo &info) { - for (idx_t col_idx = 0; col_idx < info.columns.size(); col_idx++) { - column_distinct_stats.push_back(make_uniq()); + for (const auto &column_name : info.columns) { + auto &column = info.table->GetColumn(column_name); + if (DistinctStatistics::TypeIsSupported(column.GetType())) { + column_distinct_stats.push_back(make_uniq()); + } else { + column_distinct_stats.push_back(nullptr); + } } }; @@ -30,8 +35,14 @@ unique_ptr PhysicalVacuum::GetLocalSinkState(ExecutionContext &c class VacuumGlobalSinkState : public GlobalSinkState { public: explicit VacuumGlobalSinkState(VacuumInfo &info) { - for (idx_t col_idx = 0; col_idx < info.columns.size(); col_idx++) { - column_distinct_stats.push_back(make_uniq()); + + for (const auto &column_name : info.columns) { + auto &column = info.table->GetColumn(column_name); + if (DistinctStatistics::TypeIsSupported(column.GetType())) { + column_distinct_stats.push_back(make_uniq()); + } else { + column_distinct_stats.push_back(nullptr); + } } }; @@ -58,13 +69,17 @@ SinkResultType PhysicalVacuum::Sink(ExecutionContext &context, DataChunk &chunk, } SinkCombineResultType PhysicalVacuum::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &gstate = input.global_state.Cast(); - auto &lstate = input.local_state.Cast(); + auto &g_state = input.global_state.Cast(); + auto &l_state = input.local_state.Cast(); - lock_guard lock(gstate.stats_lock); - D_ASSERT(gstate.column_distinct_stats.size() == lstate.column_distinct_stats.size()); - for (idx_t col_idx = 0; col_idx < gstate.column_distinct_stats.size(); col_idx++) { - gstate.column_distinct_stats[col_idx]->Merge(*lstate.column_distinct_stats[col_idx]); + lock_guard lock(g_state.stats_lock); + D_ASSERT(g_state.column_distinct_stats.size() == l_state.column_distinct_stats.size()); + + for (idx_t col_idx = 0; col_idx < g_state.column_distinct_stats.size(); col_idx++) { + if (g_state.column_distinct_stats[col_idx]) { + D_ASSERT(l_state.column_distinct_stats[col_idx]); + g_state.column_distinct_stats[col_idx]->Merge(*l_state.column_distinct_stats[col_idx]); + } } return SinkCombineResultType::FINISHED; diff --git a/src/duckdb/src/execution/operator/join/perfect_hash_join_executor.cpp b/src/duckdb/src/execution/operator/join/perfect_hash_join_executor.cpp index 7c7ad73e7..ff3078bcc 100644 --- a/src/duckdb/src/execution/operator/join/perfect_hash_join_executor.cpp +++ b/src/duckdb/src/execution/operator/join/perfect_hash_join_executor.cpp @@ -20,7 +20,7 @@ bool PerfectHashJoinExecutor::CanDoPerfectHashJoin() { bool PerfectHashJoinExecutor::BuildPerfectHashTable(LogicalType &key_type) { // First, allocate memory for each build column auto build_size = perfect_join_statistics.build_range + 1; - for (const auto &type : ht.build_types) { + for (const auto &type : join.rhs_output_types) { perfect_hash_table.emplace_back(type, build_size); } @@ -69,16 +69,15 @@ bool PerfectHashJoinExecutor::FullScanHashTable(LogicalType &key_type) { // Full scan the remaining build columns and fill the perfect hash table const auto build_size = perfect_join_statistics.build_range + 1; - for (idx_t i = 0; i < ht.build_types.size(); i++) { + for (idx_t i = 0; i < join.rhs_output_types.size(); i++) { auto &vector = perfect_hash_table[i]; - D_ASSERT(vector.GetType() == ht.build_types[i]); + const auto output_col_idx = ht.output_columns[i]; + D_ASSERT(vector.GetType() == ht.layout.GetTypes()[output_col_idx]); if (build_size > STANDARD_VECTOR_SIZE) { auto &col_mask = FlatVector::Validity(vector); col_mask.Initialize(build_size); } - - const auto col_no = ht.condition_types.size() + i; - data_collection.Gather(tuples_addresses, sel_tuples, key_count, col_no, vector, sel_build); + data_collection.Gather(tuples_addresses, sel_tuples, key_count, output_col_idx, vector, sel_build); } return true; @@ -189,9 +188,9 @@ OperatorResultType PerfectHashJoinExecutor::ProbePerfectHashTable(ExecutionConte result.Slice(input, state.probe_sel_vec, probe_sel_count, 0); } // on the build side, we need to fetch the data and build dictionary vectors with the sel_vec - for (idx_t i = 0; i < ht.build_types.size(); i++) { + for (idx_t i = 0; i < join.rhs_output_types.size(); i++) { auto &result_vector = result.data[input.ColumnCount() + i]; - D_ASSERT(result_vector.GetType() == ht.build_types[i]); + D_ASSERT(result_vector.GetType() == ht.layout.GetTypes()[ht.output_columns[i]]); auto &build_vec = perfect_hash_table[i]; result_vector.Reference(build_vec); result_vector.Slice(state.build_sel_vec, probe_sel_count); diff --git a/src/duckdb/src/execution/operator/join/physical_comparison_join.cpp b/src/duckdb/src/execution/operator/join/physical_comparison_join.cpp index dc4e9b976..ba0c4e5a0 100644 --- a/src/duckdb/src/execution/operator/join/physical_comparison_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_comparison_join.cpp @@ -1,5 +1,5 @@ #include "duckdb/execution/operator/join/physical_comparison_join.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/common/enum_util.hpp" namespace duckdb { diff --git a/src/duckdb/src/execution/operator/join/physical_delim_join.cpp b/src/duckdb/src/execution/operator/join/physical_delim_join.cpp index 487fc35df..5d9f2806f 100644 --- a/src/duckdb/src/execution/operator/join/physical_delim_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_delim_join.cpp @@ -1,30 +1,15 @@ #include "duckdb/execution/operator/join/physical_delim_join.hpp" -#include "duckdb/common/types/column/column_data_collection.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" -#include "duckdb/execution/operator/scan/physical_column_data_scan.hpp" -#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" -#include "duckdb/parallel/meta_pipeline.hpp" -#include "duckdb/parallel/pipeline.hpp" -#include "duckdb/parallel/thread_context.hpp" namespace duckdb { -PhysicalDelimJoin::PhysicalDelimJoin(vector types, unique_ptr original_join, +PhysicalDelimJoin::PhysicalDelimJoin(PhysicalOperatorType type, vector types, + unique_ptr original_join, vector> delim_scans, idx_t estimated_cardinality) - : PhysicalOperator(PhysicalOperatorType::DELIM_JOIN, std::move(types), estimated_cardinality), - join(std::move(original_join)), delim_scans(std::move(delim_scans)) { - D_ASSERT(join->children.size() == 2); - // now for the original join - // we take its left child, this is the side that we will duplicate eliminate - children.push_back(std::move(join->children[0])); - - // we replace it with a PhysicalColumnDataScan, that scans the ColumnDataCollection that we keep cached - // the actual chunk collection to scan will be created in the DelimJoinGlobalState - auto cached_chunk_scan = make_uniq( - children[0]->GetTypes(), PhysicalOperatorType::COLUMN_DATA_SCAN, estimated_cardinality); - join->children[0] = std::move(cached_chunk_scan); + : PhysicalOperator(type, std::move(types), estimated_cardinality), join(std::move(original_join)), + delim_scans(std::move(delim_scans)) { + D_ASSERT(type == PhysicalOperatorType::LEFT_DELIM_JOIN || type == PhysicalOperatorType::RIGHT_DELIM_JOIN); } vector> PhysicalDelimJoin::GetChildren() const { @@ -37,115 +22,8 @@ vector> PhysicalDelimJoin::GetChildren() const return result; } -//===--------------------------------------------------------------------===// -// Sink -//===--------------------------------------------------------------------===// -class DelimJoinGlobalState : public GlobalSinkState { -public: - explicit DelimJoinGlobalState(ClientContext &context, const PhysicalDelimJoin &delim_join) - : lhs_data(context, delim_join.children[0]->GetTypes()) { - D_ASSERT(delim_join.delim_scans.size() > 0); - // set up the delim join chunk to scan in the original join - auto &cached_chunk_scan = delim_join.join->children[0]->Cast(); - cached_chunk_scan.collection = &lhs_data; - } - - ColumnDataCollection lhs_data; - mutex lhs_lock; - - void Merge(ColumnDataCollection &input) { - lock_guard guard(lhs_lock); - lhs_data.Combine(input); - } -}; - -class DelimJoinLocalState : public LocalSinkState { -public: - explicit DelimJoinLocalState(ClientContext &context, const PhysicalDelimJoin &delim_join) - : lhs_data(context, delim_join.children[0]->GetTypes()) { - lhs_data.InitializeAppend(append_state); - } - - unique_ptr distinct_state; - ColumnDataCollection lhs_data; - ColumnDataAppendState append_state; - - void Append(DataChunk &input) { - lhs_data.Append(input); - } -}; - -unique_ptr PhysicalDelimJoin::GetGlobalSinkState(ClientContext &context) const { - auto state = make_uniq(context, *this); - distinct->sink_state = distinct->GetGlobalSinkState(context); - if (delim_scans.size() > 1) { - PhysicalHashAggregate::SetMultiScan(*distinct->sink_state); - } - return std::move(state); -} - -unique_ptr PhysicalDelimJoin::GetLocalSinkState(ExecutionContext &context) const { - auto state = make_uniq(context.client, *this); - state->distinct_state = distinct->GetLocalSinkState(context); - return std::move(state); -} - -SinkResultType PhysicalDelimJoin::Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const { - auto &lstate = input.local_state.Cast(); - lstate.lhs_data.Append(lstate.append_state, chunk); - OperatorSinkInput distinct_sink_input {*distinct->sink_state, *lstate.distinct_state, input.interrupt_state}; - distinct->Sink(context, chunk, distinct_sink_input); - return SinkResultType::NEED_MORE_INPUT; -} - -SinkCombineResultType PhysicalDelimJoin::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { - auto &lstate = input.local_state.Cast(); - auto &gstate = input.global_state.Cast(); - gstate.Merge(lstate.lhs_data); - - OperatorSinkCombineInput distinct_combine_input {*distinct->sink_state, *lstate.distinct_state, - input.interrupt_state}; - distinct->Combine(context, distinct_combine_input); - - return SinkCombineResultType::FINISHED; -} - -SinkFinalizeType PhysicalDelimJoin::Finalize(Pipeline &pipeline, Event &event, ClientContext &client, - OperatorSinkFinalizeInput &input) const { - // finalize the distinct HT - D_ASSERT(distinct); - - OperatorSinkFinalizeInput finalize_input {*distinct->sink_state, input.interrupt_state}; - distinct->Finalize(pipeline, event, client, finalize_input); - return SinkFinalizeType::READY; -} - string PhysicalDelimJoin::ParamsToString() const { return join->ParamsToString(); } -//===--------------------------------------------------------------------===// -// Pipeline Construction -//===--------------------------------------------------------------------===// -void PhysicalDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { - op_state.reset(); - sink_state.reset(); - - auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, *this); - child_meta_pipeline.Build(*children[0]); - - if (type == PhysicalOperatorType::DELIM_JOIN) { - // recurse into the actual join - // any pipelines in there depend on the main pipeline - // any scan of the duplicate eliminated data on the RHS depends on this pipeline - // we add an entry to the mapping of (PhysicalOperator*) -> (Pipeline*) - auto &state = meta_pipeline.GetState(); - for (auto &delim_scan : delim_scans) { - state.delim_join_dependencies.insert( - make_pair(delim_scan, reference(*child_meta_pipeline.GetBasePipeline()))); - } - join->BuildPipelines(current, meta_pipeline); - } -} - } // namespace duckdb diff --git a/src/duckdb/src/execution/operator/join/physical_hash_join.cpp b/src/duckdb/src/execution/operator/join/physical_hash_join.cpp index 038e3004f..5e72b2065 100644 --- a/src/duckdb/src/execution/operator/join/physical_hash_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_hash_join.cpp @@ -13,29 +13,63 @@ #include "duckdb/planner/expression/bound_reference_expression.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/storage_manager.hpp" +#include "duckdb/storage/temporary_memory_manager.hpp" namespace duckdb { PhysicalHashJoin::PhysicalHashJoin(LogicalOperator &op, unique_ptr left, unique_ptr right, vector cond, JoinType join_type, - const vector &left_projection_map, - const vector &right_projection_map_p, vector delim_types, - idx_t estimated_cardinality, PerfectHashJoinStats perfect_join_stats) + const vector &left_projection_map, const vector &right_projection_map, + vector delim_types, idx_t estimated_cardinality, + PerfectHashJoinStats perfect_join_stats) : PhysicalComparisonJoin(op, PhysicalOperatorType::HASH_JOIN, std::move(cond), join_type, estimated_cardinality), - right_projection_map(right_projection_map_p), delim_types(std::move(delim_types)), - perfect_join_statistics(std::move(perfect_join_stats)) { + delim_types(std::move(delim_types)), perfect_join_statistics(std::move(perfect_join_stats)) { + D_ASSERT(left_projection_map.empty()); children.push_back(std::move(left)); children.push_back(std::move(right)); - D_ASSERT(left_projection_map.empty()); - for (auto &condition : conditions) { + // Collect condition types, and which conditions are just references (so we won't duplicate them in the payload) + unordered_map build_columns_in_conditions; + for (idx_t cond_idx = 0; cond_idx < conditions.size(); cond_idx++) { + auto &condition = conditions[cond_idx]; condition_types.push_back(condition.left->return_type); + if (condition.right->GetExpressionClass() == ExpressionClass::BOUND_REF) { + build_columns_in_conditions.emplace(condition.right->Cast().index, cond_idx); + } + } + + // For ANTI, SEMI and MARK join, we only need to store the keys, so for these the payload/RHS types are empty + if (join_type == JoinType::ANTI || join_type == JoinType::SEMI || join_type == JoinType::MARK) { + return; } - // for ANTI, SEMI and MARK join, we only need to store the keys, so for these the build types are empty - if (join_type != JoinType::ANTI && join_type != JoinType::SEMI && join_type != JoinType::MARK) { - build_types = LogicalOperator::MapTypes(children[1]->GetTypes(), right_projection_map); + auto &rhs_input_types = children[1]->GetTypes(); + + // Create a projection map for the RHS (if it was empty), for convenience + auto right_projection_map_copy = right_projection_map; + if (right_projection_map_copy.empty()) { + right_projection_map_copy.reserve(rhs_input_types.size()); + for (idx_t i = 0; i < rhs_input_types.size(); i++) { + right_projection_map_copy.emplace_back(i); + } + } + + // Now fill payload expressions/types and RHS columns/types + for (auto &rhs_col : right_projection_map_copy) { + auto &rhs_col_type = rhs_input_types[rhs_col]; + + auto it = build_columns_in_conditions.find(rhs_col); + if (it == build_columns_in_conditions.end()) { + // This rhs column is not a join key + payload_column_idxs.push_back(rhs_col); + payload_types.push_back(rhs_col_type); + rhs_output_columns.push_back(condition_types.size() + payload_types.size() - 1); + } else { + // This rhs column is a join key + rhs_output_columns.push_back(it->second); + } + rhs_output_types.push_back(rhs_col_type); } } @@ -52,7 +86,8 @@ PhysicalHashJoin::PhysicalHashJoin(LogicalOperator &op, unique_ptr temporary_memory_state; //! Global HT used by the join unique_ptr hash_table; //! The perfect hash join executor (if any) @@ -95,36 +132,37 @@ class HashJoinGlobalSinkState : public GlobalSinkState { class HashJoinLocalSinkState : public LocalSinkState { public: - HashJoinLocalSinkState(const PhysicalHashJoin &op, ClientContext &context) : build_executor(context) { + HashJoinLocalSinkState(const PhysicalHashJoin &op, ClientContext &context) : join_key_executor(context) { auto &allocator = BufferAllocator::Get(context); - if (!op.right_projection_map.empty()) { - build_chunk.Initialize(allocator, op.build_types); - } + for (auto &cond : op.conditions) { - build_executor.AddExpression(*cond.right); + join_key_executor.AddExpression(*cond.right); } join_keys.Initialize(allocator, op.condition_types); - hash_table = op.InitializeHashTable(context); + if (!op.payload_types.empty()) { + payload_chunk.Initialize(allocator, op.payload_types); + } + hash_table = op.InitializeHashTable(context); hash_table->GetSinkCollection().InitializeAppendState(append_state); } public: PartitionedTupleDataAppendState append_state; - DataChunk build_chunk; + ExpressionExecutor join_key_executor; DataChunk join_keys; - ExpressionExecutor build_executor; + + DataChunk payload_chunk; //! Thread-local HT unique_ptr hash_table; }; unique_ptr PhysicalHashJoin::InitializeHashTable(ClientContext &context) const { - auto result = - make_uniq(BufferManager::GetBufferManager(context), conditions, build_types, join_type); - result->max_ht_size = double(0.6) * BufferManager::GetBufferManager(context).GetMaxMemory(); + auto result = make_uniq(BufferManager::GetBufferManager(context), conditions, payload_types, + join_type, rhs_output_columns); if (!delim_types.empty() && join_type == JoinType::MARK) { // correlated MARK join if (delim_types.size() + 1 == conditions.size()) { @@ -138,7 +176,7 @@ unique_ptr PhysicalHashJoin::InitializeHashTable(ClientContext &c // - (2) the group containing a NULL value [in which case FALSE becomes NULL] auto &info = result->correlated_mark_join_info; - vector payload_types; + vector delim_payload_types; vector correlated_aggregates; unique_ptr aggr; @@ -149,7 +187,7 @@ unique_ptr PhysicalHashJoin::InitializeHashTable(ClientContext &c aggr = function_binder.BindAggregateFunction(CountStarFun::GetFunction(), {}, nullptr, AggregateType::NON_DISTINCT); correlated_aggregates.push_back(&*aggr); - payload_types.push_back(aggr->return_type); + delim_payload_types.push_back(aggr->return_type); info.correlated_aggregates.push_back(std::move(aggr)); auto count_fun = CountFun::GetFunction(); @@ -159,15 +197,15 @@ unique_ptr PhysicalHashJoin::InitializeHashTable(ClientContext &c aggr = function_binder.BindAggregateFunction(count_fun, std::move(children), nullptr, AggregateType::NON_DISTINCT); correlated_aggregates.push_back(&*aggr); - payload_types.push_back(aggr->return_type); + delim_payload_types.push_back(aggr->return_type); info.correlated_aggregates.push_back(std::move(aggr)); auto &allocator = BufferAllocator::Get(context); info.correlated_counts = make_uniq(context, allocator, delim_types, - payload_types, correlated_aggregates); + delim_payload_types, correlated_aggregates); info.correlated_types = delim_types; info.group_chunk.Initialize(allocator, delim_types); - info.result_chunk.Initialize(allocator, payload_types); + info.result_chunk.Initialize(allocator, delim_payload_types); } } return result; @@ -186,27 +224,23 @@ SinkResultType PhysicalHashJoin::Sink(ExecutionContext &context, DataChunk &chun // resolve the join keys for the right chunk lstate.join_keys.Reset(); - lstate.build_executor.Execute(chunk, lstate.join_keys); + lstate.join_key_executor.Execute(chunk, lstate.join_keys); // build the HT auto &ht = *lstate.hash_table; - if (!right_projection_map.empty()) { - // there is a projection map: fill the build chunk with the projected columns - lstate.build_chunk.Reset(); - lstate.build_chunk.SetCardinality(chunk); - for (idx_t i = 0; i < right_projection_map.size(); i++) { - lstate.build_chunk.data[i].Reference(chunk.data[right_projection_map[i]]); - } - ht.Build(lstate.append_state, lstate.join_keys, lstate.build_chunk); - } else if (!build_types.empty()) { - // there is not a projected map: place the entire right chunk in the HT - ht.Build(lstate.append_state, lstate.join_keys, chunk); - } else { + if (payload_types.empty()) { // there are only keys: place an empty chunk in the payload - lstate.build_chunk.SetCardinality(chunk.size()); - ht.Build(lstate.append_state, lstate.join_keys, lstate.build_chunk); + lstate.payload_chunk.SetCardinality(chunk.size()); + ht.Build(lstate.append_state, lstate.join_keys, lstate.payload_chunk); + } else { + // there are payload columns + lstate.payload_chunk.Reset(); + lstate.payload_chunk.SetCardinality(chunk); + for (idx_t i = 0; i < payload_column_idxs.size(); i++) { + lstate.payload_chunk.data[i].Reference(chunk.data[payload_column_idxs[i]]); + } + ht.Build(lstate.append_state, lstate.join_keys, lstate.payload_chunk); } - return SinkResultType::NEED_MORE_INPUT; } @@ -219,7 +253,7 @@ SinkCombineResultType PhysicalHashJoin::Combine(ExecutionContext &context, Opera gstate.local_hash_tables.push_back(std::move(lstate.hash_table)); } auto &client_profiler = QueryProfiler::Get(context.client); - context.thread.profiler.Flush(*this, lstate.build_executor, "build_executor", 1); + context.thread.profiler.Flush(*this, lstate.join_key_executor, "join_key_executor", 1); client_profiler.Flush(context.thread.profiler); return SinkCombineResultType::FINISHED; @@ -322,7 +356,7 @@ class HashJoinRepartitionTask : public ExecutorTask { } TaskExecutionResult ExecuteTask(TaskExecutionMode mode) override { - local_ht.Partition(global_ht); + local_ht.Repartition(global_ht); event->FinishTask(); return TaskExecutionResult::TASK_FINISHED; } @@ -334,10 +368,10 @@ class HashJoinRepartitionTask : public ExecutorTask { JoinHashTable &local_ht; }; -class HashJoinPartitionEvent : public BasePipelineEvent { +class HashJoinRepartitionEvent : public BasePipelineEvent { public: - HashJoinPartitionEvent(Pipeline &pipeline_p, HashJoinGlobalSinkState &sink, - vector> &local_hts) + HashJoinRepartitionEvent(Pipeline &pipeline_p, HashJoinGlobalSinkState &sink, + vector> &local_hts) : BasePipelineEvent(pipeline_p), sink(sink), local_hts(local_hts) { } @@ -346,7 +380,35 @@ class HashJoinPartitionEvent : public BasePipelineEvent { public: void Schedule() override { + D_ASSERT(sink.hash_table->GetRadixBits() > JoinHashTable::INITIAL_RADIX_BITS); + + idx_t total_size = 0; + idx_t total_count = 0; + for (auto &local_ht : local_hts) { + auto &sink_collection = local_ht->GetSinkCollection(); + total_size += sink_collection.SizeInBytes(); + total_count += sink_collection.Count(); + } + auto total_blocks = (double(total_size) + Storage::BLOCK_SIZE - 1) / Storage::BLOCK_SIZE; + auto count_per_block = total_count / total_blocks; + auto blocks_per_vector = MaxValue(STANDARD_VECTOR_SIZE / count_per_block, 2); + + // Assume 8 blocks per partition per thread (4 input, 4 output) + auto partition_multiplier = + RadixPartitioning::NumberOfPartitions(sink.hash_table->GetRadixBits() - JoinHashTable::INITIAL_RADIX_BITS); + auto thread_memory = 2 * blocks_per_vector * partition_multiplier * Storage::BLOCK_SIZE; + auto repartition_threads = MaxValue(sink.temporary_memory_state->GetReservation() / thread_memory, 1); + + if (repartition_threads < local_hts.size()) { + // Limit the number of threads working on repartitioning based on our memory reservation + for (idx_t thread_idx = repartition_threads; thread_idx < local_hts.size(); thread_idx++) { + local_hts[thread_idx % repartition_threads]->Merge(*local_hts[thread_idx]); + } + local_hts.resize(repartition_threads); + } + auto &context = pipeline->GetClientContext(); + vector> partition_tasks; partition_tasks.reserve(local_hts.size()); for (auto &local_ht : local_hts) { @@ -358,7 +420,18 @@ class HashJoinPartitionEvent : public BasePipelineEvent { void FinishEvent() override { local_hts.clear(); - sink.hash_table->PrepareExternalFinalize(); + + // Minimum reservation is now the new smallest partition size + const auto num_partitions = RadixPartitioning::NumberOfPartitions(sink.hash_table->GetRadixBits()); + vector partition_sizes(num_partitions, 0); + vector partition_counts(num_partitions, 0); + idx_t max_partition_size; + idx_t max_partition_count; + sink.hash_table->GetTotalSize(partition_sizes, partition_counts, max_partition_size, max_partition_count); + sink.temporary_memory_state->SetMinimumReservation(max_partition_size + + JoinHashTable::PointerTableSize(max_partition_count)); + + sink.hash_table->PrepareExternalFinalize(sink.temporary_memory_state->GetReservation()); sink.ScheduleFinalize(*pipeline, *this); } }; @@ -368,23 +441,36 @@ SinkFinalizeType PhysicalHashJoin::Finalize(Pipeline &pipeline, Event &event, Cl auto &sink = input.global_state.Cast(); auto &ht = *sink.hash_table; - sink.external = ht.RequiresExternalJoin(context.config, sink.local_hash_tables); + idx_t max_partition_size; + idx_t max_partition_count; + auto const total_size = ht.GetTotalSize(sink.local_hash_tables, max_partition_size, max_partition_count); + sink.temporary_memory_state->SetRemainingSize(context, total_size); + + sink.external = sink.temporary_memory_state->GetReservation() < total_size; if (sink.external) { + const auto max_partition_ht_size = max_partition_size + JoinHashTable::PointerTableSize(max_partition_count); + // External Hash Join sink.perfect_join_executor.reset(); - if (ht.RequiresPartitioning(context.config, sink.local_hash_tables)) { - auto new_event = make_shared(pipeline, sink, sink.local_hash_tables); + if (max_partition_ht_size > sink.temporary_memory_state->GetReservation()) { + // We have to repartition + ht.SetRepartitionRadixBits(sink.local_hash_tables, sink.temporary_memory_state->GetReservation(), + max_partition_size, max_partition_count); + auto new_event = make_shared(pipeline, sink, sink.local_hash_tables); event.InsertEvent(std::move(new_event)); } else { + // No repartitioning! + sink.temporary_memory_state->SetMinimumReservation(max_partition_ht_size); for (auto &local_ht : sink.local_hash_tables) { ht.Merge(*local_ht); } sink.local_hash_tables.clear(); - sink.hash_table->PrepareExternalFinalize(); + sink.hash_table->PrepareExternalFinalize(sink.temporary_memory_state->GetReservation()); sink.ScheduleFinalize(pipeline, event); } sink.finalized = true; return SinkFinalizeType::READY; } else { + // In-memory Hash Join for (auto &local_ht : sink.local_hash_tables) { ht.Merge(*local_ht); } @@ -685,9 +771,13 @@ void HashJoinGlobalSourceState::PrepareBuild(HashJoinGlobalSinkState &sink) { D_ASSERT(global_stage != HashJoinSourceStage::BUILD); auto &ht = *sink.hash_table; + // Update remaining size + sink.temporary_memory_state->SetRemainingSize(sink.context, ht.GetRemainingSize()); + // Try to put the next partitions in the block collection of the HT - if (!sink.external || !ht.PrepareExternalFinalize()) { + if (!sink.external || !ht.PrepareExternalFinalize(sink.temporary_memory_state->GetReservation())) { global_stage = HashJoinSourceStage::DONE; + sink.temporary_memory_state->SetRemainingSize(sink.context, 0); return; } @@ -904,7 +994,12 @@ SourceResultType PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk auto &lstate = input.local_state.Cast(); sink.scanned_data = true; - if (!sink.external && !(PropagatesBuildSide(join_type))) { + if (!sink.external && !PropagatesBuildSide(join_type)) { + lock_guard guard(gstate.lock); + if (gstate.global_stage != HashJoinSourceStage::DONE) { + gstate.global_stage = HashJoinSourceStage::DONE; + sink.temporary_memory_state->SetRemainingSize(context.client, 0); + } return SourceResultType::FINISHED; } @@ -926,4 +1021,34 @@ SourceResultType PhysicalHashJoin::GetData(ExecutionContext &context, DataChunk return chunk.size() == 0 ? SourceResultType::FINISHED : SourceResultType::HAVE_MORE_OUTPUT; } +double PhysicalHashJoin::GetProgress(ClientContext &context, GlobalSourceState &gstate_p) const { + auto &sink = sink_state->Cast(); + auto &gstate = gstate_p.Cast(); + + if (!sink.external) { + if (PropagatesBuildSide(join_type)) { + return double(gstate.full_outer_chunk_done) / double(gstate.full_outer_chunk_count) * 100.0; + } + return 100.0; + } + + double num_partitions = RadixPartitioning::NumberOfPartitions(sink.hash_table->GetRadixBits()); + double partition_start = sink.hash_table->GetPartitionStart(); + double partition_end = sink.hash_table->GetPartitionEnd(); + + // This many partitions are fully done + auto progress = partition_start / double(num_partitions); + + double probe_chunk_done = gstate.probe_chunk_done; + double probe_chunk_count = gstate.probe_chunk_count; + if (probe_chunk_count != 0) { + // Progress of the current round of probing, weighed by the number of partitions + auto probe_progress = double(probe_chunk_done) / double(probe_chunk_count); + // Add it to the progress, weighed by the number of partitions in the current round + progress += (partition_end - partition_start) / num_partitions * probe_progress; + } + + return progress * 100.0; +} + } // namespace duckdb diff --git a/src/duckdb/src/execution/operator/join/physical_iejoin.cpp b/src/duckdb/src/execution/operator/join/physical_iejoin.cpp index deb587814..df7fc232f 100644 --- a/src/duckdb/src/execution/operator/join/physical_iejoin.cpp +++ b/src/duckdb/src/execution/operator/join/physical_iejoin.cpp @@ -1037,8 +1037,8 @@ void PhysicalIEJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeli children[0]->BuildPipelines(*lhs_pipeline, child_meta_pipeline); // Build out RHS - auto rhs_pipeline = child_meta_pipeline.CreatePipeline(); - children[1]->BuildPipelines(*rhs_pipeline, child_meta_pipeline); + auto &rhs_pipeline = child_meta_pipeline.CreatePipeline(); + children[1]->BuildPipelines(rhs_pipeline, child_meta_pipeline); // Despite having the same sink, RHS and everything created after it need their own (same) PipelineFinishEvent child_meta_pipeline.AddFinishEvent(rhs_pipeline); diff --git a/src/duckdb/src/execution/operator/join/physical_join.cpp b/src/duckdb/src/execution/operator/join/physical_join.cpp index 46d4535ee..6d9813c9e 100644 --- a/src/duckdb/src/execution/operator/join/physical_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_join.cpp @@ -28,7 +28,8 @@ bool PhysicalJoin::EmptyResultIfRHSIsEmpty() const { //===--------------------------------------------------------------------===// // Pipeline Construction //===--------------------------------------------------------------------===// -void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, PhysicalOperator &op) { +void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline, PhysicalOperator &op, + bool build_rhs) { op.op_state.reset(); op.sink_state.reset(); @@ -39,11 +40,13 @@ void PhysicalJoin::BuildJoinPipelines(Pipeline ¤t, MetaPipeline &meta_pipe // save the last added pipeline to set up dependencies later (in case we need to add a child pipeline) vector> pipelines_so_far; meta_pipeline.GetPipelines(pipelines_so_far, false); - auto last_pipeline = pipelines_so_far.back().get(); + auto &last_pipeline = *pipelines_so_far.back(); - // on the RHS (build side), we construct a child MetaPipeline with this operator as its sink - auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, op); - child_meta_pipeline.Build(*op.children[1]); + if (build_rhs) { + // on the RHS (build side), we construct a child MetaPipeline with this operator as its sink + auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, op); + child_meta_pipeline.Build(*op.children[1]); + } // continue building the current pipeline on the LHS (probe side) op.children[0]->BuildPipelines(current, meta_pipeline); diff --git a/src/duckdb/src/execution/operator/join/physical_left_delim_join.cpp b/src/duckdb/src/execution/operator/join/physical_left_delim_join.cpp new file mode 100644 index 000000000..04a6ce80d --- /dev/null +++ b/src/duckdb/src/execution/operator/join/physical_left_delim_join.cpp @@ -0,0 +1,137 @@ +#include "duckdb/execution/operator/join/physical_left_delim_join.hpp" + +#include "duckdb/common/types/column/column_data_collection.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" +#include "duckdb/execution/operator/scan/physical_column_data_scan.hpp" +#include "duckdb/parallel/meta_pipeline.hpp" +#include "duckdb/parallel/pipeline.hpp" +#include "duckdb/parallel/thread_context.hpp" + +namespace duckdb { + +PhysicalLeftDelimJoin::PhysicalLeftDelimJoin(vector types, unique_ptr original_join, + vector> delim_scans, + idx_t estimated_cardinality) + : PhysicalDelimJoin(PhysicalOperatorType::LEFT_DELIM_JOIN, std::move(types), std::move(original_join), + std::move(delim_scans), estimated_cardinality) { + D_ASSERT(join->children.size() == 2); + // now for the original join + // we take its left child, this is the side that we will duplicate eliminate + children.push_back(std::move(join->children[0])); + + // we replace it with a PhysicalColumnDataScan, that scans the ColumnDataCollection that we keep cached + // the actual chunk collection to scan will be created in the LeftDelimJoinGlobalState + auto cached_chunk_scan = make_uniq( + children[0]->GetTypes(), PhysicalOperatorType::COLUMN_DATA_SCAN, estimated_cardinality); + join->children[0] = std::move(cached_chunk_scan); +} + +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// +class LeftDelimJoinGlobalState : public GlobalSinkState { +public: + explicit LeftDelimJoinGlobalState(ClientContext &context, const PhysicalLeftDelimJoin &delim_join) + : lhs_data(context, delim_join.children[0]->GetTypes()) { + D_ASSERT(!delim_join.delim_scans.empty()); + // set up the delim join chunk to scan in the original join + auto &cached_chunk_scan = delim_join.join->children[0]->Cast(); + cached_chunk_scan.collection = &lhs_data; + } + + ColumnDataCollection lhs_data; + mutex lhs_lock; + + void Merge(ColumnDataCollection &input) { + lock_guard guard(lhs_lock); + lhs_data.Combine(input); + } +}; + +class LeftDelimJoinLocalState : public LocalSinkState { +public: + explicit LeftDelimJoinLocalState(ClientContext &context, const PhysicalLeftDelimJoin &delim_join) + : lhs_data(context, delim_join.children[0]->GetTypes()) { + lhs_data.InitializeAppend(append_state); + } + + unique_ptr distinct_state; + ColumnDataCollection lhs_data; + ColumnDataAppendState append_state; + + void Append(DataChunk &input) { + lhs_data.Append(input); + } +}; + +unique_ptr PhysicalLeftDelimJoin::GetGlobalSinkState(ClientContext &context) const { + auto state = make_uniq(context, *this); + distinct->sink_state = distinct->GetGlobalSinkState(context); + if (delim_scans.size() > 1) { + PhysicalHashAggregate::SetMultiScan(*distinct->sink_state); + } + return std::move(state); +} + +unique_ptr PhysicalLeftDelimJoin::GetLocalSinkState(ExecutionContext &context) const { + auto state = make_uniq(context.client, *this); + state->distinct_state = distinct->GetLocalSinkState(context); + return std::move(state); +} + +SinkResultType PhysicalLeftDelimJoin::Sink(ExecutionContext &context, DataChunk &chunk, + OperatorSinkInput &input) const { + auto &lstate = input.local_state.Cast(); + lstate.lhs_data.Append(lstate.append_state, chunk); + OperatorSinkInput distinct_sink_input {*distinct->sink_state, *lstate.distinct_state, input.interrupt_state}; + distinct->Sink(context, chunk, distinct_sink_input); + return SinkResultType::NEED_MORE_INPUT; +} + +SinkCombineResultType PhysicalLeftDelimJoin::Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const { + auto &lstate = input.local_state.Cast(); + auto &gstate = input.global_state.Cast(); + gstate.Merge(lstate.lhs_data); + + OperatorSinkCombineInput distinct_combine_input {*distinct->sink_state, *lstate.distinct_state, + input.interrupt_state}; + distinct->Combine(context, distinct_combine_input); + + return SinkCombineResultType::FINISHED; +} + +SinkFinalizeType PhysicalLeftDelimJoin::Finalize(Pipeline &pipeline, Event &event, ClientContext &client, + OperatorSinkFinalizeInput &input) const { + // finalize the distinct HT + D_ASSERT(distinct); + + OperatorSinkFinalizeInput finalize_input {*distinct->sink_state, input.interrupt_state}; + distinct->Finalize(pipeline, event, client, finalize_input); + return SinkFinalizeType::READY; +} + +//===--------------------------------------------------------------------===// +// Pipeline Construction +//===--------------------------------------------------------------------===// +void PhysicalLeftDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { + op_state.reset(); + sink_state.reset(); + + auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, *this); + child_meta_pipeline.Build(*children[0]); + + D_ASSERT(type == PhysicalOperatorType::LEFT_DELIM_JOIN); + // recurse into the actual join + // any pipelines in there depend on the main pipeline + // any scan of the duplicate eliminated data on the RHS depends on this pipeline + // we add an entry to the mapping of (PhysicalOperator*) -> (Pipeline*) + auto &state = meta_pipeline.GetState(); + for (auto &delim_scan : delim_scans) { + state.delim_join_dependencies.insert( + make_pair(delim_scan, reference(*child_meta_pipeline.GetBasePipeline()))); + } + join->BuildPipelines(current, meta_pipeline); +} + +} // namespace duckdb 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 d1f0a5cfb..60e4eb2ff 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 @@ -118,6 +118,12 @@ bool PhysicalNestedLoopJoin::IsSupported(const vector &conditions return false; } } + // To avoid situations like https://github.com/duckdb/duckdb/issues/10046 + // If there is an equality in the conditions, a hash join is planned + // with one condition, we can use mark join logic, otherwise we should use physical blockwise nl join + if (join_type == JoinType::SEMI || join_type == JoinType::ANTI) { + return conditions.size() == 1; + } return true; } diff --git a/src/duckdb/src/execution/operator/join/physical_range_join.cpp b/src/duckdb/src/execution/operator/join/physical_range_join.cpp index 07cbc327e..a87ffd2ba 100644 --- a/src/duckdb/src/execution/operator/join/physical_range_join.cpp +++ b/src/duckdb/src/execution/operator/join/physical_range_join.cpp @@ -5,6 +5,8 @@ #include "duckdb/common/row_operations/row_operations.hpp" #include "duckdb/common/sort/comparators.hpp" #include "duckdb/common/sort/sort.hpp" +#include "duckdb/common/types/validity_mask.hpp" +#include "duckdb/common/types/vector.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/main/client_context.hpp" @@ -40,13 +42,15 @@ void PhysicalRangeJoin::LocalSortedTable::Sink(DataChunk &input, GlobalSortState keys.Reset(); executor.Execute(input, keys); + // Do not operate on primary key directly to avoid modifying the input chunk + Vector primary = keys.data[0]; // Count the NULLs so we can exclude them later - has_null += MergeNulls(op.conditions); + has_null += MergeNulls(primary, op.conditions); count += keys.size(); // Only sort the primary key DataChunk join_head; - join_head.data.emplace_back(keys.data[0]); + join_head.data.emplace_back(primary); join_head.SetCardinality(keys.size()); // Sink the data into the local sort state @@ -214,7 +218,7 @@ PhysicalRangeJoin::PhysicalRangeJoin(LogicalComparisonJoin &op, PhysicalOperator unprojected_types.insert(unprojected_types.end(), types.begin(), types.end()); } -idx_t PhysicalRangeJoin::LocalSortedTable::MergeNulls(const vector &conditions) { +idx_t PhysicalRangeJoin::LocalSortedTable::MergeNulls(Vector &primary, const vector &conditions) { // Merge the validity masks of the comparison keys into the primary // Return the number of NULLs in the resulting chunk D_ASSERT(keys.ColumnCount() > 0); @@ -227,11 +231,18 @@ idx_t PhysicalRangeJoin::LocalSortedTable::MergeNulls(const vector types, unique_ptr original_join, + vector> delim_scans, + idx_t estimated_cardinality) + : PhysicalDelimJoin(PhysicalOperatorType::RIGHT_DELIM_JOIN, std::move(types), std::move(original_join), + std::move(delim_scans), estimated_cardinality) { + D_ASSERT(join->children.size() == 2); + // now for the original join + // we take its right child, this is the side that we will duplicate eliminate + children.push_back(std::move(join->children[1])); + + // we replace it with a PhysicalDummyScan, which contains no data, just the types, it won't be scanned anyway + join->children[1] = make_uniq(children[0]->GetTypes(), estimated_cardinality); +} + +//===--------------------------------------------------------------------===// +// Sink +//===--------------------------------------------------------------------===// +class RightDelimJoinGlobalState : public GlobalSinkState {}; + +class RightDelimJoinLocalState : public LocalSinkState { +public: + unique_ptr join_state; + unique_ptr distinct_state; +}; + +unique_ptr PhysicalRightDelimJoin::GetGlobalSinkState(ClientContext &context) const { + auto state = make_uniq(); + join->sink_state = join->GetGlobalSinkState(context); + distinct->sink_state = distinct->GetGlobalSinkState(context); + if (delim_scans.size() > 1) { + PhysicalHashAggregate::SetMultiScan(*distinct->sink_state); + } + return std::move(state); +} + +unique_ptr PhysicalRightDelimJoin::GetLocalSinkState(ExecutionContext &context) const { + auto state = make_uniq(); + state->join_state = join->GetLocalSinkState(context); + state->distinct_state = distinct->GetLocalSinkState(context); + return std::move(state); +} + +SinkResultType PhysicalRightDelimJoin::Sink(ExecutionContext &context, DataChunk &chunk, + OperatorSinkInput &input) const { + auto &lstate = input.local_state.Cast(); + + OperatorSinkInput join_sink_input {*join->sink_state, *lstate.join_state, input.interrupt_state}; + join->Sink(context, chunk, join_sink_input); + + OperatorSinkInput distinct_sink_input {*distinct->sink_state, *lstate.distinct_state, input.interrupt_state}; + distinct->Sink(context, chunk, distinct_sink_input); + + return SinkResultType::NEED_MORE_INPUT; +} + +SinkCombineResultType PhysicalRightDelimJoin::Combine(ExecutionContext &context, + OperatorSinkCombineInput &input) const { + auto &lstate = input.local_state.Cast(); + + OperatorSinkCombineInput join_combine_input {*join->sink_state, *lstate.join_state, input.interrupt_state}; + join->Combine(context, join_combine_input); + + OperatorSinkCombineInput distinct_combine_input {*distinct->sink_state, *lstate.distinct_state, + input.interrupt_state}; + distinct->Combine(context, distinct_combine_input); + + return SinkCombineResultType::FINISHED; +} + +SinkFinalizeType PhysicalRightDelimJoin::Finalize(Pipeline &pipeline, Event &event, ClientContext &client, + OperatorSinkFinalizeInput &input) const { + D_ASSERT(join); + D_ASSERT(distinct); + + OperatorSinkFinalizeInput join_finalize_input {*join->sink_state, input.interrupt_state}; + join->Finalize(pipeline, event, client, join_finalize_input); + + OperatorSinkFinalizeInput distinct_finalize_input {*distinct->sink_state, input.interrupt_state}; + distinct->Finalize(pipeline, event, client, distinct_finalize_input); + + return SinkFinalizeType::READY; +} + +//===--------------------------------------------------------------------===// +// Pipeline Construction +//===--------------------------------------------------------------------===// +void PhysicalRightDelimJoin::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) { + op_state.reset(); + sink_state.reset(); + + auto &child_meta_pipeline = meta_pipeline.CreateChildMetaPipeline(current, *this); + child_meta_pipeline.Build(*children[0]); + + D_ASSERT(type == PhysicalOperatorType::RIGHT_DELIM_JOIN); + // recurse into the actual join + // any pipelines in there depend on the main pipeline + // any scan of the duplicate eliminated data on the LHS depends on this pipeline + // we add an entry to the mapping of (PhysicalOperator*) -> (Pipeline*) + auto &state = meta_pipeline.GetState(); + for (auto &delim_scan : delim_scans) { + state.delim_join_dependencies.insert( + make_pair(delim_scan, reference(*child_meta_pipeline.GetBasePipeline()))); + } + + // Build join pipelines without building the RHS (already built in the Sink of this op) + PhysicalJoin::BuildJoinPipelines(current, meta_pipeline, *join, false); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/operator/persistent/physical_copy_to_file.cpp b/src/duckdb/src/execution/operator/persistent/physical_copy_to_file.cpp index abe2ed281..57bc9bbc0 100644 --- a/src/duckdb/src/execution/operator/persistent/physical_copy_to_file.cpp +++ b/src/duckdb/src/execution/operator/persistent/physical_copy_to_file.cpp @@ -1,10 +1,12 @@ #include "duckdb/execution/operator/persistent/physical_copy_to_file.hpp" -#include "duckdb/common/vector_operations/vector_operations.hpp" -#include "duckdb/common/hive_partitioning.hpp" -#include "duckdb/common/file_system.hpp" + #include "duckdb/common/file_opener.hpp" -#include "duckdb/common/types/uuid.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/hive_partitioning.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/types/uuid.hpp" +#include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/storage/storage_lock.hpp" #include @@ -15,9 +17,9 @@ class CopyToFunctionGlobalState : public GlobalSinkState { explicit CopyToFunctionGlobalState(unique_ptr global_state) : rows_copied(0), last_file_offset(0), global_state(std::move(global_state)) { } - mutex lock; - idx_t rows_copied; - idx_t last_file_offset; + StorageLock lock; + atomic rows_copied; + atomic last_file_offset; unique_ptr global_state; idx_t created_directories = 0; @@ -40,10 +42,74 @@ class CopyToFunctionLocalState : public LocalSinkState { idx_t writer_offset; }; +unique_ptr PhysicalCopyToFile::CreateFileState(ClientContext &context, + GlobalSinkState &sink) const { + auto &g = sink.Cast(); + idx_t this_file_offset = g.last_file_offset++; + auto &fs = FileSystem::GetFileSystem(context); + string output_path(filename_pattern.CreateFilename(fs, file_path, file_extension, this_file_offset)); + if (fs.FileExists(output_path) && !overwrite_or_ignore) { + throw IOException("%s exists! Enable OVERWRITE_OR_IGNORE option to force writing", output_path); + } + return function.copy_to_initialize_global(context, *bind_data, output_path); +} + +unique_ptr PhysicalCopyToFile::GetLocalSinkState(ExecutionContext &context) const { + if (partition_output) { + auto &g = sink_state->Cast(); + + auto state = make_uniq(nullptr); + state->writer_offset = g.last_file_offset++; + state->part_buffer = + make_uniq(context.client, expected_types, partition_columns, g.partition_state); + state->part_buffer_append_state = make_uniq(); + state->part_buffer->InitializeAppendState(*state->part_buffer_append_state); + return std::move(state); + } + auto res = make_uniq(function.copy_to_initialize_local(context, *bind_data)); + if (per_thread_output) { + res->global_state = CreateFileState(context.client, *sink_state); + } + return std::move(res); +} + +unique_ptr PhysicalCopyToFile::GetGlobalSinkState(ClientContext &context) const { + + if (partition_output || per_thread_output || file_size_bytes.IsValid()) { + auto &fs = FileSystem::GetFileSystem(context); + + if (fs.FileExists(file_path) && !overwrite_or_ignore) { + throw IOException("%s exists! Enable OVERWRITE_OR_IGNORE option to force writing", file_path); + } + if (!fs.DirectoryExists(file_path)) { + fs.CreateDirectory(file_path); + } else if (!overwrite_or_ignore) { + idx_t n_files = 0; + fs.ListFiles(file_path, [&n_files](const string &path, bool) { n_files++; }); + if (n_files > 0) { + throw IOException("Directory %s is not empty! Enable OVERWRITE_OR_IGNORE option to force writing", + file_path); + } + } + + auto state = make_uniq(nullptr); + if (!per_thread_output && file_size_bytes.IsValid()) { + state->global_state = CreateFileState(context, *state); + } + + if (partition_output) { + state->partition_state = make_shared(); + } + + return std::move(state); + } + + return make_uniq(function.copy_to_initialize_global(context, *bind_data, file_path)); +} + //===--------------------------------------------------------------------===// // Sink //===--------------------------------------------------------------------===// - void PhysicalCopyToFile::MoveTmpFile(ClientContext &context, const string &tmp_file_path) { auto &fs = FileSystem::GetFileSystem(context); auto file_path = tmp_file_path.substr(0, tmp_file_path.length() - 4); @@ -68,12 +134,39 @@ SinkResultType PhysicalCopyToFile::Sink(ExecutionContext &context, DataChunk &ch return SinkResultType::NEED_MORE_INPUT; } - { - lock_guard glock(g.lock); - g.rows_copied += chunk.size(); + g.rows_copied += chunk.size(); + + if (per_thread_output) { + auto &gstate = l.global_state; + function.copy_to_sink(context, *bind_data, *gstate, *l.local_state, chunk); + + if (file_size_bytes.IsValid() && function.file_size_bytes(*gstate) > file_size_bytes.GetIndex()) { + function.copy_to_finalize(context.client, *bind_data, *gstate); + gstate = CreateFileState(context.client, *sink_state); + } + return SinkResultType::NEED_MORE_INPUT; + } + + if (!file_size_bytes.IsValid()) { + function.copy_to_sink(context, *bind_data, *g.global_state, *l.local_state, chunk); + return SinkResultType::NEED_MORE_INPUT; + } + + // FILE_SIZE_BYTES is set, but threads write to the same file, synchronize using lock + auto &gstate = g.global_state; + auto lock = g.lock.GetExclusiveLock(); + if (function.file_size_bytes(*gstate) > file_size_bytes.GetIndex()) { + auto owned_gstate = std::move(gstate); + gstate = CreateFileState(context.client, *sink_state); + lock.reset(); + function.copy_to_finalize(context.client, *bind_data, *owned_gstate); + } else { + lock.reset(); } - function.copy_to_sink(context, *bind_data, per_thread_output ? *l.global_state : *g.global_state, *l.local_state, - chunk); + + lock = g.lock.GetSharedLock(); + function.copy_to_sink(context, *bind_data, *gstate, *l.local_state, chunk); + return SinkResultType::NEED_MORE_INPUT; } @@ -121,7 +214,7 @@ SinkCombineResultType PhysicalCopyToFile::Combine(ExecutionContext &context, Ope StringUtil::RTrim(trimmed_path, fs.PathSeparator(trimmed_path)); { // create directories - lock_guard global_lock(g.lock); + auto lock = g.lock.GetExclusiveLock(); lock_guard global_lock_on_partition_state(g.partition_state->lock); const auto &global_partitions = g.partition_state->partitions; // global_partitions have partitions added only at the back, so it's fine to only traverse the last part @@ -134,10 +227,10 @@ SinkCombineResultType PhysicalCopyToFile::Combine(ExecutionContext &context, Ope for (idx_t i = 0; i < partitions.size(); i++) { string hive_path = GetDirectory(partition_columns, names, partition_key_map[i]->values, trimmed_path, fs); - string full_path(filename_pattern.CreateFilename(fs, hive_path, function.extension, l.writer_offset)); + string full_path(filename_pattern.CreateFilename(fs, hive_path, file_extension, l.writer_offset)); if (fs.FileExists(full_path) && !overwrite_or_ignore) { - throw IOException("failed to create " + full_path + - ", file exists! Enable OVERWRITE_OR_IGNORE option to force writing"); + throw IOException( + "failed to create %s, file exists! Enable OVERWRITE_OR_IGNORE option to force writing", full_path); } // Create a writer for the current file auto fun_data_global = function.copy_to_initialize_global(context.client, *bind_data, full_path); @@ -150,16 +243,17 @@ SinkCombineResultType PhysicalCopyToFile::Combine(ExecutionContext &context, Ope function.copy_to_combine(context, *bind_data, *fun_data_global, *fun_data_local); function.copy_to_finalize(context.client, *bind_data, *fun_data_global); } - - return SinkCombineResultType::FINISHED; - } - - if (function.copy_to_combine) { - function.copy_to_combine(context, *bind_data, per_thread_output ? *l.global_state : *g.global_state, - *l.local_state); - + } else if (function.copy_to_combine) { if (per_thread_output) { + // For PER_THREAD_OUTPUT, we can combine/finalize immediately + function.copy_to_combine(context, *bind_data, *l.global_state, *l.local_state); function.copy_to_finalize(context.client, *bind_data, *l.global_state); + } else if (file_size_bytes.IsValid()) { + // File in global state may change with FILE_SIZE_BYTES, need to grab lock + auto lock = g.lock.GetSharedLock(); + function.copy_to_combine(context, *bind_data, *g.global_state, *l.local_state); + } else { + function.copy_to_combine(context, *bind_data, *g.global_state, *l.local_state); } } @@ -177,78 +271,15 @@ SinkFinalizeType PhysicalCopyToFile::Finalize(Pipeline &pipeline, Event &event, function.copy_to_finalize(context, *bind_data, *gstate.global_state); if (use_tmp_file) { - D_ASSERT(!per_thread_output); // FIXME - D_ASSERT(!partition_output); // FIXME + D_ASSERT(!per_thread_output); + D_ASSERT(!partition_output); + D_ASSERT(!file_size_bytes.IsValid()); MoveTmpFile(context, file_path); } } return SinkFinalizeType::READY; } -unique_ptr PhysicalCopyToFile::GetLocalSinkState(ExecutionContext &context) const { - if (partition_output) { - auto state = make_uniq(nullptr); - { - auto &g = sink_state->Cast(); - lock_guard glock(g.lock); - state->writer_offset = g.last_file_offset++; - - state->part_buffer = make_uniq(context.client, expected_types, partition_columns, - g.partition_state); - state->part_buffer_append_state = make_uniq(); - state->part_buffer->InitializeAppendState(*state->part_buffer_append_state); - } - return std::move(state); - } - auto res = make_uniq(function.copy_to_initialize_local(context, *bind_data)); - if (per_thread_output) { - idx_t this_file_offset; - { - auto &g = sink_state->Cast(); - lock_guard glock(g.lock); - this_file_offset = g.last_file_offset++; - } - auto &fs = FileSystem::GetFileSystem(context.client); - string output_path(filename_pattern.CreateFilename(fs, file_path, function.extension, this_file_offset)); - if (fs.FileExists(output_path) && !overwrite_or_ignore) { - throw IOException("%s exists! Enable OVERWRITE_OR_IGNORE option to force writing", output_path); - } - res->global_state = function.copy_to_initialize_global(context.client, *bind_data, output_path); - } - return std::move(res); -} - -unique_ptr PhysicalCopyToFile::GetGlobalSinkState(ClientContext &context) const { - - if (partition_output || per_thread_output) { - auto &fs = FileSystem::GetFileSystem(context); - - if (fs.FileExists(file_path) && !overwrite_or_ignore) { - throw IOException("%s exists! Enable OVERWRITE_OR_IGNORE option to force writing", file_path); - } - if (!fs.DirectoryExists(file_path)) { - fs.CreateDirectory(file_path); - } else if (!overwrite_or_ignore) { - idx_t n_files = 0; - fs.ListFiles(file_path, [&n_files](const string &path, bool) { n_files++; }); - if (n_files > 0) { - throw IOException("Directory %s is not empty! Enable OVERWRITE_OR_IGNORE option to force writing", - file_path); - } - } - - auto state = make_uniq(nullptr); - - if (partition_output) { - state->partition_state = make_shared(); - } - - return std::move(state); - } - - return make_uniq(function.copy_to_initialize_global(context, *bind_data, file_path)); -} - //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/execution/operator/projection/physical_unnest.cpp b/src/duckdb/src/execution/operator/projection/physical_unnest.cpp index d5632f80e..356b22724 100644 --- a/src/duckdb/src/execution/operator/projection/physical_unnest.cpp +++ b/src/duckdb/src/execution/operator/projection/physical_unnest.cpp @@ -1,5 +1,6 @@ #include "duckdb/execution/operator/projection/physical_unnest.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/common/algorithm.hpp" #include "duckdb/execution/expression_executor.hpp" @@ -163,6 +164,9 @@ static void UnnestVector(UnifiedVectorFormat &child_vector_data, Vector &child_v case PhysicalType::UINT64: TemplatedUnnest(child_vector_data, start, end, result); break; + case PhysicalType::UINT128: + TemplatedUnnest(child_vector_data, start, end, result); + break; case PhysicalType::FLOAT: TemplatedUnnest(child_vector_data, start, end, result); break; diff --git a/src/duckdb/src/execution/operator/scan/physical_column_data_scan.cpp b/src/duckdb/src/execution/operator/scan/physical_column_data_scan.cpp index 55c00b7a6..d91e7a5c3 100644 --- a/src/duckdb/src/execution/operator/scan/physical_column_data_scan.cpp +++ b/src/duckdb/src/execution/operator/scan/physical_column_data_scan.cpp @@ -1,7 +1,8 @@ #include "duckdb/execution/operator/scan/physical_column_data_scan.hpp" -#include "duckdb/execution/operator/join/physical_delim_join.hpp" +#include "duckdb/common/types/column/column_data_collection.hpp" #include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" +#include "duckdb/execution/operator/join/physical_delim_join.hpp" #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/parallel/pipeline.hpp" @@ -14,6 +15,11 @@ PhysicalColumnDataScan::PhysicalColumnDataScan(vector types, Physic owned_collection(std::move(owned_collection_p)) { } +PhysicalColumnDataScan::PhysicalColumnDataScan(vector types, PhysicalOperatorType op_type, + idx_t estimated_cardinality, idx_t cte_index) + : PhysicalOperator(op_type, std::move(types), estimated_cardinality), collection(nullptr), cte_index(cte_index) { +} + class PhysicalColumnDataScanState : public GlobalSourceState { public: explicit PhysicalColumnDataScanState() : initialized(false) { @@ -58,7 +64,8 @@ void PhysicalColumnDataScan::BuildPipelines(Pipeline ¤t, MetaPipeline &met auto delim_dependency = entry->second.get().shared_from_this(); auto delim_sink = state.GetPipelineSink(*delim_dependency); D_ASSERT(delim_sink); - D_ASSERT(delim_sink->type == PhysicalOperatorType::DELIM_JOIN); + D_ASSERT(delim_sink->type == PhysicalOperatorType::LEFT_DELIM_JOIN || + delim_sink->type == PhysicalOperatorType::RIGHT_DELIM_JOIN); auto &delim_join = delim_sink->Cast(); current.AddDependency(delim_dependency); state.SetPipelineSource(current, delim_join.distinct->Cast()); diff --git a/src/duckdb/src/execution/operator/schema/physical_alter.cpp b/src/duckdb/src/execution/operator/schema/physical_alter.cpp index 7fc00006c..0e73fc29d 100644 --- a/src/duckdb/src/execution/operator/schema/physical_alter.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_alter.cpp @@ -1,5 +1,6 @@ #include "duckdb/execution/operator/schema/physical_alter.hpp" #include "duckdb/parser/parsed_data/alter_table_info.hpp" +#include "duckdb/main/database_manager.hpp" #include "duckdb/catalog/catalog.hpp" namespace duckdb { diff --git a/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp b/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp index 1ed983e56..e7325405f 100644 --- a/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_create_art_index.cpp @@ -9,6 +9,7 @@ #include "duckdb/storage/index.hpp" #include "duckdb/storage/storage_manager.hpp" #include "duckdb/storage/table/append_state.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -177,7 +178,7 @@ SinkFinalizeType PhysicalCreateARTIndex::Finalize(Pipeline &pipeline, Event &eve auto &index = index_entry->Cast(); index.initial_index_size = state.global_index->GetInMemorySize(); - index.info = storage.info; + index.info = make_shared(storage.info, index.name); for (auto &parsed_expr : info->parsed_expressions) { index.parsed_expressions.push_back(parsed_expr->Copy()); } diff --git a/src/duckdb/src/execution/operator/schema/physical_create_schema.cpp b/src/duckdb/src/execution/operator/schema/physical_create_schema.cpp index d5e340b87..b0b031390 100644 --- a/src/duckdb/src/execution/operator/schema/physical_create_schema.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_create_schema.cpp @@ -1,5 +1,6 @@ #include "duckdb/execution/operator/schema/physical_create_schema.hpp" #include "duckdb/catalog/catalog.hpp" +#include "duckdb/common/exception/binder_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/execution/operator/schema/physical_drop.cpp b/src/duckdb/src/execution/operator/schema/physical_drop.cpp index 36cc3976c..cfb6841a8 100644 --- a/src/duckdb/src/execution/operator/schema/physical_drop.cpp +++ b/src/duckdb/src/execution/operator/schema/physical_drop.cpp @@ -3,6 +3,7 @@ #include "duckdb/main/database_manager.hpp" #include "duckdb/main/database.hpp" #include "duckdb/main/client_context.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/catalog/catalog_search_path.hpp" #include "duckdb/main/settings.hpp" @@ -38,6 +39,15 @@ SourceResultType PhysicalDrop::GetData(ExecutionContext &context, DataChunk &chu } break; } + case CatalogType::SECRET_ENTRY: { + // Note: the schema param is used to optionally pass the storage to drop from + D_ASSERT(info->extra_drop_info); + auto &extra_info = info->extra_drop_info->Cast(); + SecretManager::Get(context.client) + .DropSecretByName(context.client, info->name, info->if_not_found, extra_info.persist_mode, + extra_info.secret_storage); + break; + } default: { auto &catalog = Catalog::GetCatalog(context.client, info->catalog); catalog.DropEntry(context.client, *info); diff --git a/src/duckdb/src/execution/operator/set/physical_union.cpp b/src/duckdb/src/execution/operator/set/physical_union.cpp index a25343b6f..81e5b49ee 100644 --- a/src/duckdb/src/execution/operator/set/physical_union.cpp +++ b/src/duckdb/src/execution/operator/set/physical_union.cpp @@ -41,7 +41,7 @@ void PhysicalUnion::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipelin } // create a union pipeline that is identical to 'current' - auto union_pipeline = meta_pipeline.CreateUnionPipeline(current, order_matters); + auto &union_pipeline = meta_pipeline.CreateUnionPipeline(current, order_matters); // continue with the current pipeline children[0]->BuildPipelines(current, meta_pipeline); @@ -52,7 +52,7 @@ void PhysicalUnion::BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipelin } // build the union pipeline - children[1]->BuildPipelines(*union_pipeline, meta_pipeline); + children[1]->BuildPipelines(union_pipeline, meta_pipeline); // Assign proper batch index to the union pipeline // This needs to happen after the pipelines have been built because unions can be nested diff --git a/src/duckdb/src/execution/physical_operator.cpp b/src/duckdb/src/execution/physical_operator.cpp index 2cd57fbb6..ba5ba3e22 100644 --- a/src/duckdb/src/execution/physical_operator.cpp +++ b/src/duckdb/src/execution/physical_operator.cpp @@ -10,6 +10,7 @@ #include "duckdb/parallel/pipeline.hpp" #include "duckdb/parallel/thread_context.hpp" #include "duckdb/storage/buffer_manager.hpp" +#include "duckdb/storage/buffer/buffer_pool.hpp" namespace duckdb { @@ -121,7 +122,7 @@ unique_ptr PhysicalOperator::GetGlobalSinkState(ClientContext & idx_t PhysicalOperator::GetMaxThreadMemory(ClientContext &context) { // Memory usage per thread should scale with max mem / num threads // We take 1/4th of this, to be conservative - idx_t max_memory = BufferManager::GetBufferManager(context).GetMaxMemory(); + idx_t max_memory = BufferManager::GetBufferManager(context).GetQueryMaxMemory(); idx_t num_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); return (max_memory / num_threads) / 4; } 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 33ea2484e..166bc9449 100644 --- a/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_comparison_join.cpp @@ -74,7 +74,8 @@ void CheckForPerfectJoinOpt(LogicalComparisonJoin &op, PerfectHashJoinStats &joi // with integral internal types for (auto &&join_stat : op.join_stats) { if (!TypeIsInteger(join_stat->GetType().InternalType()) || - join_stat->GetType().InternalType() == PhysicalType::INT128) { + join_stat->GetType().InternalType() == PhysicalType::INT128 || + join_stat->GetType().InternalType() == PhysicalType::UINT128) { // perfect join not possible for non-integral types or hugeint return; } diff --git a/src/duckdb/src/execution/physical_plan/plan_copy_to_file.cpp b/src/duckdb/src/execution/physical_plan/plan_copy_to_file.cpp index 0fe227512..373fbdfcf 100644 --- a/src/duckdb/src/execution/physical_plan/plan_copy_to_file.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_copy_to_file.cpp @@ -1,7 +1,7 @@ -#include "duckdb/execution/physical_plan_generator.hpp" -#include "duckdb/execution/operator/persistent/physical_copy_to_file.hpp" #include "duckdb/execution/operator/persistent/physical_batch_copy_to_file.hpp" +#include "duckdb/execution/operator/persistent/physical_copy_to_file.hpp" #include "duckdb/execution/operator/persistent/physical_fixed_batch_copy.hpp" +#include "duckdb/execution/physical_plan_generator.hpp" #include "duckdb/planner/operator/logical_copy_to_file.hpp" namespace duckdb { @@ -15,7 +15,8 @@ unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalCopyToFile if (op.use_tmp_file) { op.file_path += ".tmp"; } - if (op.per_thread_output || op.partition_output || !op.partition_columns.empty() || op.overwrite_or_ignore) { + if (op.per_thread_output || op.file_size_bytes.IsValid() || op.partition_output || !op.partition_columns.empty() || + op.overwrite_or_ignore) { // hive-partitioning/per-thread output does not care about insertion order, and does not support batch indexes preserve_insertion_order = false; supports_batch_index = false; @@ -51,7 +52,11 @@ unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalCopyToFile copy->use_tmp_file = op.use_tmp_file; copy->overwrite_or_ignore = op.overwrite_or_ignore; copy->filename_pattern = op.filename_pattern; + copy->file_extension = op.file_extension; copy->per_thread_output = op.per_thread_output; + if (op.file_size_bytes.IsValid()) { + copy->file_size_bytes = op.file_size_bytes; + } copy->partition_output = op.partition_output; copy->partition_columns = op.partition_columns; copy->names = op.names; diff --git a/src/duckdb/src/execution/physical_plan/plan_create_index.cpp b/src/duckdb/src/execution/physical_plan/plan_create_index.cpp index 4d9e101f6..abfcd6289 100644 --- a/src/duckdb/src/execution/physical_plan/plan_create_index.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_create_index.cpp @@ -25,7 +25,7 @@ unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalCreateInde // because they make deletions and lookups unfeasible for (idx_t i = 0; i < op.unbound_expressions.size(); i++) { auto &expr = op.unbound_expressions[i]; - if (expr->HasSideEffects()) { + if (!expr->IsConsistent()) { throw BinderException("Index keys cannot contain expressions with side effects."); } } @@ -33,7 +33,7 @@ unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalCreateInde // if we get here and the index type is not ART, we throw an exception // because we don't support any other index type yet. However, an operator extension could have // replaced this part of the plan with a different index creation operator. - if (op.info->index_type != "ART") { + if (op.info->index_type != ART::TYPE_NAME) { throw BinderException("Unknown index type: " + op.info->index_type); } diff --git a/src/duckdb/src/execution/physical_plan/plan_create_secret.cpp b/src/duckdb/src/execution/physical_plan/plan_create_secret.cpp new file mode 100644 index 000000000..b4818c809 --- /dev/null +++ b/src/duckdb/src/execution/physical_plan/plan_create_secret.cpp @@ -0,0 +1,11 @@ +#include "duckdb/execution/physical_plan_generator.hpp" +#include "duckdb/planner/operator/logical_create_secret.hpp" +#include "duckdb/execution/operator/helper/physical_create_secret.hpp" + +namespace duckdb { + +unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalCreateSecret &op) { + return make_uniq(op.info, op.estimated_cardinality); +} + +} // namespace duckdb diff --git a/src/duckdb/src/execution/physical_plan/plan_create_table.cpp b/src/duckdb/src/execution/physical_plan/plan_create_table.cpp index 880559a14..be854e0da 100644 --- a/src/duckdb/src/execution/physical_plan/plan_create_table.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_create_table.cpp @@ -1,17 +1,17 @@ #include "duckdb/catalog/catalog_entry/scalar_function_catalog_entry.hpp" +#include "duckdb/catalog/catalog_entry/schema_catalog_entry.hpp" #include "duckdb/catalog/catalog_entry/table_catalog_entry.hpp" +#include "duckdb/catalog/duck_catalog.hpp" +#include "duckdb/execution/operator/persistent/physical_batch_insert.hpp" +#include "duckdb/execution/operator/persistent/physical_insert.hpp" #include "duckdb/execution/operator/schema/physical_create_table.hpp" #include "duckdb/execution/physical_plan_generator.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/parser/parsed_data/create_table_info.hpp" -#include "duckdb/execution/operator/persistent/physical_insert.hpp" +#include "duckdb/planner/constraints/bound_check_constraint.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckdb/planner/operator/logical_create_table.hpp" -#include "duckdb/main/config.hpp" -#include "duckdb/execution/operator/persistent/physical_batch_insert.hpp" -#include "duckdb/planner/constraints/bound_check_constraint.hpp" -#include "duckdb/parallel/task_scheduler.hpp" -#include "duckdb/catalog/duck_catalog.hpp" -#include "duckdb/catalog/catalog_entry/schema_catalog_entry.hpp" namespace duckdb { @@ -22,10 +22,10 @@ unique_ptr DuckCatalog::PlanCreateTableAs(ClientContext &conte auto num_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); unique_ptr create; if (!parallel_streaming_insert && use_batch_index) { - create = make_uniq(op, op.schema, std::move(op.info), op.estimated_cardinality); + create = make_uniq(op, op.schema, std::move(op.info), 0); } else { - create = make_uniq(op, op.schema, std::move(op.info), op.estimated_cardinality, + create = make_uniq(op, op.schema, std::move(op.info), 0, parallel_streaming_insert && num_threads > 1); } diff --git a/src/duckdb/src/execution/physical_plan/plan_delim_join.cpp b/src/duckdb/src/execution/physical_plan/plan_delim_join.cpp index f30cb2591..ba4246b52 100644 --- a/src/duckdb/src/execution/physical_plan/plan_delim_join.cpp +++ b/src/duckdb/src/execution/physical_plan/plan_delim_join.cpp @@ -1,9 +1,11 @@ -#include "duckdb/execution/operator/join/physical_delim_join.hpp" +#include "duckdb/common/enum_util.hpp" +#include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" #include "duckdb/execution/operator/join/physical_hash_join.hpp" +#include "duckdb/execution/operator/join/physical_left_delim_join.hpp" +#include "duckdb/execution/operator/join/physical_right_delim_join.hpp" #include "duckdb/execution/operator/projection/physical_projection.hpp" #include "duckdb/execution/physical_plan_generator.hpp" #include "duckdb/planner/expression/bound_reference_expression.hpp" -#include "duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp" namespace duckdb { @@ -22,11 +24,12 @@ unique_ptr PhysicalPlanGenerator::PlanDelimJoin(LogicalCompari // this should create a join, not a cross product D_ASSERT(plan && plan->type != PhysicalOperatorType::CROSS_PRODUCT); // duplicate eliminated join - // first gather the scans on the duplicate eliminated data set from the RHS + // first gather the scans on the duplicate eliminated data set from the delim side + const idx_t delim_idx = op.delim_flipped ? 0 : 1; vector> delim_scans; - GatherDelimScans(*plan->children[1], delim_scans); + GatherDelimScans(*plan->children[delim_idx], delim_scans); if (delim_scans.empty()) { - // no duplicate eliminated scans in the RHS! + // no duplicate eliminated scans in the delim side! // in this case we don't need to create a delim join // just push the normal join return plan; @@ -40,7 +43,13 @@ unique_ptr PhysicalPlanGenerator::PlanDelimJoin(LogicalCompari distinct_groups.push_back(make_uniq(bound_ref.return_type, bound_ref.index)); } // now create the duplicate eliminated join - auto delim_join = make_uniq(op.types, std::move(plan), delim_scans, op.estimated_cardinality); + unique_ptr delim_join; + if (op.delim_flipped) { + delim_join = + make_uniq(op.types, std::move(plan), delim_scans, op.estimated_cardinality); + } else { + delim_join = make_uniq(op.types, std::move(plan), delim_scans, op.estimated_cardinality); + } // we still have to create the DISTINCT clause that is used to generate the duplicate eliminated chunk delim_join->distinct = make_uniq(context, delim_types, std::move(distinct_expressions), std::move(distinct_groups), op.estimated_cardinality); diff --git a/src/duckdb/src/execution/physical_plan/plan_show_select.cpp b/src/duckdb/src/execution/physical_plan/plan_show_select.cpp deleted file mode 100644 index a7392f961..000000000 --- a/src/duckdb/src/execution/physical_plan/plan_show_select.cpp +++ /dev/null @@ -1,47 +0,0 @@ -#include "duckdb/execution/operator/scan/physical_column_data_scan.hpp" -#include "duckdb/execution/physical_plan_generator.hpp" -#include "duckdb/parser/parsed_data/show_select_info.hpp" -#include "duckdb/planner/operator/logical_show.hpp" - -namespace duckdb { - -unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalShow &op) { - DataChunk output; - output.Initialize(Allocator::Get(context), op.types); - - auto collection = make_uniq(context, op.types); - ColumnDataAppendState append_state; - collection->InitializeAppend(append_state); - for (idx_t column_idx = 0; column_idx < op.types_select.size(); column_idx++) { - auto type = op.types_select[column_idx]; - auto &name = op.aliases[column_idx]; - - // "name", TypeId::VARCHAR - output.SetValue(0, output.size(), Value(name)); - // "type", TypeId::VARCHAR - output.SetValue(1, output.size(), Value(type.ToString())); - // "null", TypeId::VARCHAR - output.SetValue(2, output.size(), Value("YES")); - // "pk", TypeId::BOOL - output.SetValue(3, output.size(), Value()); - // "dflt_value", TypeId::VARCHAR - output.SetValue(4, output.size(), Value()); - // "extra", TypeId::VARCHAR - output.SetValue(5, output.size(), Value()); - - output.SetCardinality(output.size() + 1); - if (output.size() == STANDARD_VECTOR_SIZE) { - collection->Append(append_state, output); - output.Reset(); - } - } - - collection->Append(append_state, output); - - // create a chunk scan to output the result - auto chunk_scan = make_uniq(op.types, PhysicalOperatorType::COLUMN_DATA_SCAN, - op.estimated_cardinality, std::move(collection)); - return std::move(chunk_scan); -} - -} // namespace duckdb diff --git a/src/duckdb/src/execution/physical_plan_generator.cpp b/src/duckdb/src/execution/physical_plan_generator.cpp index 685008447..705d522ae 100644 --- a/src/duckdb/src/execution/physical_plan_generator.cpp +++ b/src/duckdb/src/execution/physical_plan_generator.cpp @@ -153,12 +153,12 @@ unique_ptr PhysicalPlanGenerator::CreatePlan(LogicalOperator & case LogicalOperatorType::LOGICAL_CREATE_INDEX: plan = CreatePlan(op.Cast()); break; + case LogicalOperatorType::LOGICAL_CREATE_SECRET: + plan = CreatePlan(op.Cast()); + break; case LogicalOperatorType::LOGICAL_EXPLAIN: plan = CreatePlan(op.Cast()); break; - case LogicalOperatorType::LOGICAL_SHOW: - plan = CreatePlan(op.Cast()); - break; case LogicalOperatorType::LOGICAL_DISTINCT: plan = CreatePlan(op.Cast()); break; diff --git a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp index cf0f480d5..52bb30562 100644 --- a/src/duckdb/src/execution/radix_partitioned_hashtable.cpp +++ b/src/duckdb/src/execution/radix_partitioned_hashtable.cpp @@ -10,6 +10,7 @@ #include "duckdb/main/config.hpp" #include "duckdb/parallel/event.hpp" #include "duckdb/planner/expression/bound_reference_expression.hpp" +#include "duckdb/storage/temporary_memory_manager.hpp" namespace duckdb { @@ -69,9 +70,11 @@ unique_ptr RadixPartitionedHashTable::CreateHT(Client // Sink //===--------------------------------------------------------------------===// struct AggregatePartition { - explicit AggregatePartition(unique_ptr data_p) : data(std::move(data_p)), finalized(false) { + explicit AggregatePartition(unique_ptr data_p) + : data(std::move(data_p)), progress(0), finalized(false) { } unique_ptr data; + atomic progress; atomic finalized; }; @@ -135,6 +138,10 @@ class RadixHTGlobalSinkState : public GlobalSinkState { void Destroy(); public: + ClientContext &context; + //! Temporary memory state for managing this hash table's memory usage + unique_ptr temporary_memory_state; + //! The radix HT const RadixPartitionedHashTable &radix_ht; //! Config for partitioning @@ -166,12 +173,29 @@ class RadixHTGlobalSinkState : public GlobalSinkState { TupleDataPinProperties scan_pin_properties; //! Total count before combining idx_t count_before_combining; + //! Maximum partition size if all unique + idx_t max_partition_size; }; -RadixHTGlobalSinkState::RadixHTGlobalSinkState(ClientContext &context, const RadixPartitionedHashTable &radix_ht_p) - : radix_ht(radix_ht_p), config(context, *this), finalized(false), external(false), active_threads(0), +RadixHTGlobalSinkState::RadixHTGlobalSinkState(ClientContext &context_p, const RadixPartitionedHashTable &radix_ht_p) + : context(context_p), temporary_memory_state(TemporaryMemoryManager::Get(context).Register(context)), + radix_ht(radix_ht_p), config(context, *this), finalized(false), external(false), active_threads(0), any_combined(false), finalize_idx(0), scan_pin_properties(TupleDataPinProperties::DESTROY_AFTER_DONE), - count_before_combining(0) { + count_before_combining(0), max_partition_size(0) { + + auto tuples_per_block = Storage::BLOCK_ALLOC_SIZE / radix_ht.GetLayout().GetRowWidth(); + idx_t ht_count = config.sink_capacity / GroupedAggregateHashTable::LOAD_FACTOR; + auto num_partitions = RadixPartitioning::NumberOfPartitions(config.GetRadixBits()); + auto count_per_partition = ht_count / num_partitions; + auto blocks_per_partition = (count_per_partition + tuples_per_block) / tuples_per_block + 1; + auto ht_size = blocks_per_partition * Storage::BLOCK_ALLOC_SIZE + config.sink_capacity * sizeof(aggr_ht_entry_t); + + // This really is the minimum reservation that we can do + idx_t num_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); + auto minimum_reservation = num_threads * ht_size; + + temporary_memory_state->SetMinimumReservation(minimum_reservation); + temporary_memory_state->SetRemainingSize(context, minimum_reservation); } RadixHTGlobalSinkState::~RadixHTGlobalSinkState() { @@ -326,10 +350,27 @@ bool MaybeRepartition(ClientContext &context, RadixHTGlobalSinkState &gstate, Ra auto &partitioned_data = ht.GetPartitionedData(); // Check if we're approaching the memory limit - const idx_t n_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); - const idx_t limit = BufferManager::GetBufferManager(context).GetMaxMemory(); - const idx_t thread_limit = 0.6 * limit / n_threads; - if (ht.GetPartitionedData()->SizeInBytes() > thread_limit || context.config.force_external) { + auto &temporary_memory_state = *gstate.temporary_memory_state; + const auto total_size = ht.GetPartitionedData()->SizeInBytes() + ht.Capacity() * sizeof(aggr_ht_entry_t); + idx_t thread_limit = temporary_memory_state.GetReservation() / gstate.active_threads; + if (total_size > thread_limit) { + // We're over the thread memory limit + if (!gstate.external) { + // We haven't yet triggered out-of-core behavior, but maybe we don't have to, grab the lock and check again + lock_guard guard(gstate.lock); + thread_limit = temporary_memory_state.GetReservation() / gstate.active_threads; + if (total_size > thread_limit) { + // Out-of-core would be triggered below, try to increase the reservation, but only up to a limit + auto new_remaining_size = + MinValue(2 * temporary_memory_state.GetRemainingSize(), + BufferManager::GetBufferManager(context).GetQueryMaxMemory() / 2); + temporary_memory_state.SetRemainingSize(context, new_remaining_size); + thread_limit = temporary_memory_state.GetReservation() / gstate.active_threads; + } + } + } + + if (total_size > thread_limit) { if (gstate.config.SetRadixBitsToExternal()) { // We're approaching the memory limit, unpin the data if (!lstate.abandoned_data) { @@ -449,7 +490,7 @@ void RadixPartitionedHashTable::Combine(ExecutionContext &context, GlobalSinkSta gstate.stored_allocators.emplace_back(ht.GetAggregateAllocator()); } -void RadixPartitionedHashTable::Finalize(ClientContext &, GlobalSinkState &gstate_p) const { +void RadixPartitionedHashTable::Finalize(ClientContext &context, GlobalSinkState &gstate_p) const { auto &gstate = gstate_p.Cast(); if (gstate.uncombined_data) { @@ -463,7 +504,13 @@ void RadixPartitionedHashTable::Finalize(ClientContext &, GlobalSinkState &gstat const auto n_partitions = uncombined_partition_data.size(); gstate.partitions.reserve(n_partitions); for (idx_t i = 0; i < n_partitions; i++) { - gstate.partitions.emplace_back(make_uniq(std::move(uncombined_partition_data[i]))); + auto &partition = uncombined_partition_data[i]; + auto partition_size = + partition->SizeInBytes() + + GroupedAggregateHashTable::GetCapacityForCount(partition->Count()) * sizeof(aggr_ht_entry_t); + gstate.max_partition_size = MaxValue(gstate.max_partition_size, partition_size); + + gstate.partitions.emplace_back(make_uniq(std::move(partition))); if (single_ht) { gstate.finalize_idx++; gstate.partitions.back()->finalized = true; @@ -473,15 +520,30 @@ void RadixPartitionedHashTable::Finalize(ClientContext &, GlobalSinkState &gstat gstate.count_before_combining = 0; } + // Minimum of combining one partition at a time + gstate.temporary_memory_state->SetMinimumReservation(gstate.max_partition_size); + // Maximum of combining all partitions + auto max_threads = + MinValue(TaskScheduler::GetScheduler(context).NumberOfThreads(), gstate.partitions.size()); + gstate.temporary_memory_state->SetRemainingSize(context, max_threads * gstate.max_partition_size); + gstate.finalized = true; } //===--------------------------------------------------------------------===// // Source //===--------------------------------------------------------------------===// -idx_t RadixPartitionedHashTable::NumberOfPartitions(GlobalSinkState &sink_p) const { +idx_t RadixPartitionedHashTable::MaxThreads(GlobalSinkState &sink_p) const { auto &sink = sink_p.Cast(); - return sink.partitions.size(); + if (sink.partitions.empty()) { + return 0; + } + + // This many partitions will fit given our reservation (at least 1)) + auto partitions_fit = MaxValue(sink.temporary_memory_state->GetReservation() / sink.max_partition_size, 1); + + // Of course, limit it to the number of actual partitions + return MinValue(sink.partitions.size(), partitions_fit); } void RadixPartitionedHashTable::SetMultiScan(GlobalSinkState &sink_p) { @@ -654,8 +716,9 @@ void RadixHTLocalSourceState::Finalize(RadixHTGlobalSinkState &sink, RadixHTGlob const idx_t n_threads = TaskScheduler::GetScheduler(gstate.context).NumberOfThreads(); const idx_t memory_limit = BufferManager::GetBufferManager(gstate.context).GetMaxMemory(); const idx_t thread_limit = 0.6 * memory_limit / n_threads; - const auto size_per_entry = partition.data->GetLayout().GetRowWidth() + - GroupedAggregateHashTable::LOAD_FACTOR * sizeof(aggr_ht_entry_t); + + const idx_t size_per_entry = partition.data->SizeInBytes() / partition.data->Count() + + idx_t(GroupedAggregateHashTable::LOAD_FACTOR * sizeof(aggr_ht_entry_t)); const auto capacity_limit = NextPowerOfTwo(thread_limit / size_per_entry); ht = sink.radix_ht.CreateHT(gstate.context, MinValue(capacity, capacity_limit), 0); @@ -667,7 +730,7 @@ void RadixHTLocalSourceState::Finalize(RadixHTGlobalSinkState &sink, RadixHTGlob } // Now combine the uncombined data using this thread's HT - ht->Combine(*partition.data); + ht->Combine(*partition.data, &partition.progress); ht->UnpinData(); // Move the combined data back to the partition @@ -771,6 +834,7 @@ SourceResultType RadixPartitionedHashTable::GetData(ExecutionContext &context, D sink.scan_pin_properties == TupleDataPinProperties::DESTROY_AFTER_DONE); if (gstate.finished) { + sink.temporary_memory_state->SetRemainingSize(context.client, 0); return SourceResultType::FINISHED; } @@ -821,4 +885,25 @@ SourceResultType RadixPartitionedHashTable::GetData(ExecutionContext &context, D } } +double RadixPartitionedHashTable::GetProgress(ClientContext &, GlobalSinkState &sink_p, + GlobalSourceState &gstate_p) const { + auto &sink = sink_p.Cast(); + auto &gstate = gstate_p.Cast(); + + // Get partition combine progress, weigh it 2x + double total_progress = 0; + for (auto &partition : sink.partitions) { + total_progress += partition->progress * 2.0; + } + + // Get scan progress, weigh it 1x + total_progress += gstate.scan_done; + + // Divide by 3x for the weights, and the number of partitions to get a value between 0 and 1 again + total_progress /= 3.0 * sink.partitions.size(); + + // Multiply by 100 to get a percentage + return 100.0 * total_progress; +} + } // namespace duckdb diff --git a/src/duckdb/src/execution/reservoir_sample.cpp b/src/duckdb/src/execution/reservoir_sample.cpp index 3522b71a1..8344a7301 100644 --- a/src/duckdb/src/execution/reservoir_sample.cpp +++ b/src/duckdb/src/execution/reservoir_sample.cpp @@ -1,34 +1,51 @@ #include "duckdb/execution/reservoir_sample.hpp" +#include "duckdb/common/types/data_chunk.hpp" #include "duckdb/common/pair.hpp" namespace duckdb { +void BlockingSample::Serialize(Serializer &serializer) const { +} + +unique_ptr BlockingSample::Deserialize(Deserializer &deserializer) { + return nullptr; +} + ReservoirSample::ReservoirSample(Allocator &allocator, idx_t sample_count, int64_t seed) - : BlockingSample(seed), sample_count(sample_count), reservoir(allocator) { + : BlockingSample(seed), allocator(allocator), sample_count(sample_count), reservoir_initialized(false) { } void ReservoirSample::AddToReservoir(DataChunk &input) { if (sample_count == 0) { + // sample count is 0, means no samples were requested return; } + base_reservoir_sample.num_entries_seen_total += input.size(); // Input: A population V of n weighted items // Output: A reservoir R with a size m // 1: The first m items of V are inserted into R // first we need to check if the reservoir already has "m" elements - if (reservoir.Count() < sample_count) { + if (!reservoir_chunk || reservoir_chunk->size() < sample_count) { if (FillReservoir(input) == 0) { // entire chunk was consumed by reservoir return; } } - // find the position of next_index relative to current_count + D_ASSERT(reservoir_chunk); + D_ASSERT(reservoir_chunk->size() == sample_count); + // Initialize the weights if they have not been already + if (base_reservoir_sample.reservoir_weights.empty()) { + base_reservoir_sample.InitializeReservoir(reservoir_chunk->size(), sample_count); + } + // find the position of next_index_to_sample relative to number of seen entries (num_entries_to_skip_b4_next_sample) idx_t remaining = input.size(); idx_t base_offset = 0; while (true) { - idx_t offset = base_reservoir_sample.next_index - base_reservoir_sample.current_count; + idx_t offset = + base_reservoir_sample.next_index_to_sample - base_reservoir_sample.num_entries_to_skip_b4_next_sample; if (offset >= remaining) { // not in this chunk! increment current count and go to the next chunk - base_reservoir_sample.current_count += remaining; + base_reservoir_sample.num_entries_to_skip_b4_next_sample += remaining; return; } // in this chunk! replace the element @@ -40,39 +57,77 @@ void ReservoirSample::AddToReservoir(DataChunk &input) { } unique_ptr ReservoirSample::GetChunk() { - return reservoir.Fetch(); + if (!reservoir_chunk || reservoir_chunk->size() == 0) { + return nullptr; + } + auto collected_sample_count = reservoir_chunk->size(); + if (collected_sample_count > STANDARD_VECTOR_SIZE) { + // get from the back to avoid creating two selection vectors + // one to return the first STANDARD_VECTOR_SIZE + // another to replace the reservoir_chunk with the first STANDARD VECTOR SIZE missing + auto ret = make_uniq(); + auto samples_remaining = collected_sample_count - STANDARD_VECTOR_SIZE; + auto reservoir_types = reservoir_chunk->GetTypes(); + SelectionVector sel(STANDARD_VECTOR_SIZE); + for (idx_t i = samples_remaining; i < collected_sample_count; i++) { + sel.set_index(i - samples_remaining, i); + } + ret->Initialize(allocator, reservoir_types.begin(), reservoir_types.end(), STANDARD_VECTOR_SIZE); + ret->Slice(*reservoir_chunk, sel, STANDARD_VECTOR_SIZE); + ret->SetCardinality(STANDARD_VECTOR_SIZE); + // reduce capacity and cardinality of the sample data chunk + reservoir_chunk->SetCardinality(samples_remaining); + return ret; + } + return std::move(reservoir_chunk); } -void ReservoirSample::ReplaceElement(DataChunk &input, idx_t index_in_chunk) { +void ReservoirSample::ReplaceElement(DataChunk &input, idx_t index_in_chunk, double with_weight) { // replace the entry in the reservoir // 8. The item in R with the minimum key is replaced by item vi + D_ASSERT(input.ColumnCount() == reservoir_chunk->ColumnCount()); for (idx_t col_idx = 0; col_idx < input.ColumnCount(); col_idx++) { - reservoir.SetValue(col_idx, base_reservoir_sample.min_entry, input.GetValue(col_idx, index_in_chunk)); + reservoir_chunk->SetValue(col_idx, base_reservoir_sample.min_weighted_entry_index, + input.GetValue(col_idx, index_in_chunk)); } - base_reservoir_sample.ReplaceElement(); + base_reservoir_sample.ReplaceElement(with_weight); +} + +void ReservoirSample::InitializeReservoir(DataChunk &input) { + reservoir_chunk = make_uniq(); + reservoir_chunk->Initialize(allocator, input.GetTypes(), sample_count); + for (idx_t col_idx = 0; col_idx < reservoir_chunk->ColumnCount(); col_idx++) { + FlatVector::Validity(reservoir_chunk->data[col_idx]).Initialize(sample_count); + } + reservoir_initialized = true; } idx_t ReservoirSample::FillReservoir(DataChunk &input) { idx_t chunk_count = input.size(); input.Flatten(); + auto num_added_samples = reservoir_chunk ? reservoir_chunk->size() : 0; + D_ASSERT(num_added_samples <= sample_count); - // we have not: append to the reservoir + // required count is what we still need to add to the reservoir idx_t required_count; - if (reservoir.Count() + chunk_count >= sample_count) { + if (num_added_samples + chunk_count >= sample_count) { // have to limit the count of the chunk - required_count = sample_count - reservoir.Count(); + required_count = sample_count - num_added_samples; } else { // we copy the entire chunk required_count = chunk_count; } - // instead of copying we just change the pointer in the current chunk input.SetCardinality(required_count); - reservoir.Append(input); - base_reservoir_sample.InitializeReservoir(reservoir.Count(), sample_count); + // initialize the reservoir + if (!reservoir_initialized) { + InitializeReservoir(input); + } + reservoir_chunk->Append(input, false, nullptr, required_count); + base_reservoir_sample.InitializeReservoir(required_count, sample_count); - // check if there are still elements remaining - // this happens if we are on a boundary + // check if there are still elements remaining in the Input data chunk that should be + // randomly sampled and potentially added. This happens if we are on a boundary // for example, input.size() is 1024, but our sample size is 10 if (required_count == chunk_count) { // we are done here @@ -89,6 +144,10 @@ idx_t ReservoirSample::FillReservoir(DataChunk &input) { return input.size(); } +void ReservoirSample::Finalize() { + return; +} + ReservoirSamplePercentage::ReservoirSamplePercentage(Allocator &allocator, double percentage, int64_t seed) : BlockingSample(seed), allocator(allocator), sample_percentage(percentage / 100.0), current_count(0), is_finalized(false) { @@ -97,6 +156,7 @@ ReservoirSamplePercentage::ReservoirSamplePercentage(Allocator &allocator, doubl } void ReservoirSamplePercentage::AddToReservoir(DataChunk &input) { + base_reservoir_sample.num_entries_seen_total += input.size(); if (current_count + input.size() > RESERVOIR_THRESHOLD) { // we don't have enough space in our current reservoir // first check what we still need to append to the current sample @@ -120,8 +180,9 @@ void ReservoirSamplePercentage::AddToReservoir(DataChunk &input) { if (append_to_next_sample > 0) { // slice the input for the remainder SelectionVector sel(append_to_next_sample); - for (idx_t i = 0; i < append_to_next_sample; i++) { - sel.set_index(i, append_to_current_sample_count + i); + for (idx_t i = append_to_current_sample_count; i < append_to_next_sample + append_to_current_sample_count; + i++) { + sel.set_index(i - append_to_current_sample_count, i); } input.Slice(sel, append_to_next_sample); } @@ -159,7 +220,16 @@ unique_ptr ReservoirSamplePercentage::GetChunk() { void ReservoirSamplePercentage::Finalize() { // need to finalize the current sample, if any - if (current_count > 0) { + // we are finializing, so we are starting to return chunks. Our last chunk has + // sample_percentage * RESERVOIR_THRESHOLD entries that hold samples. + // if our current count is less than the sample_percentage * RESERVOIR_THRESHOLD + // then we have sampled too much for the current_sample and we need to redo the sample + // otherwise we can just push the current sample back + // Imagine sampling 70% of 100 rows (so 70 rows). We allocate sample_percentage * RESERVOIR_THRESHOLD + // ----------------------------------------- + auto sampled_more_than_required = + current_count > sample_percentage * RESERVOIR_THRESHOLD || finished_samples.empty(); + if (current_count > 0 && sampled_more_than_required) { // create a new sample auto new_sample_size = idx_t(round(sample_percentage * current_count)); auto new_sample = make_uniq(allocator, new_sample_size, random.NextRandomInteger()); @@ -171,15 +241,20 @@ void ReservoirSamplePercentage::Finalize() { new_sample->AddToReservoir(*chunk); } finished_samples.push_back(std::move(new_sample)); + } else { + finished_samples.push_back(std::move(current_sample)); } + // when finalizing, current_sample is null. All samples are now in finished samples. + current_sample = nullptr; is_finalized = true; } BaseReservoirSampling::BaseReservoirSampling(int64_t seed) : random(seed) { - next_index = 0; - min_threshold = 0; - min_entry = 0; - current_count = 0; + next_index_to_sample = 0; + min_weight_threshold = 0; + min_weighted_entry_index = 0; + num_entries_to_skip_b4_next_sample = 0; + num_entries_seen_total = 0; } BaseReservoirSampling::BaseReservoirSampling() : BaseReservoirSampling(-1) { @@ -209,23 +284,28 @@ void BaseReservoirSampling::SetNextEntry() { //! 5. From the current item vc skip items until item vi , such that: //! 6. wc +wc+1 +···+wi−1 < Xw <= wc +wc+1 +···+wi−1 +wi //! since all our weights are 1 (uniform sampling), we can just determine the amount of elements to skip - min_threshold = t_w; - min_entry = min_key.second; - next_index = MaxValue(1, idx_t(round(x_w))); - current_count = 0; + min_weight_threshold = t_w; + min_weighted_entry_index = min_key.second; + next_index_to_sample = MaxValue(1, idx_t(round(x_w))); + num_entries_to_skip_b4_next_sample = 0; } -void BaseReservoirSampling::ReplaceElement() { +void BaseReservoirSampling::ReplaceElement(double with_weight) { //! replace the entry in the reservoir //! pop the minimum entry reservoir_weights.pop(); //! now update the reservoir //! 8. Let tw = Tw i , r2 = random(tw,1) and vi’s key: ki = (r2)1/wi //! 9. The new threshold Tw is the new minimum key of R - //! we generate a random number between (min_threshold, 1) - double r2 = random.NextRandom(min_threshold, 1); + //! we generate a random number between (min_weight_threshold, 1) + double r2 = random.NextRandom(min_weight_threshold, 1); + + //! if we are merging two reservoir samples use the weight passed + if (with_weight >= 0) { + r2 = with_weight; + } //! now we insert the new weight into the reservoir - reservoir_weights.emplace(-r2, min_entry); + reservoir_weights.push(std::make_pair(-r2, min_weighted_entry_index)); //! we update the min entry with the new min entry in the reservoir SetNextEntry(); } diff --git a/src/duckdb/src/execution/window_executor.cpp b/src/duckdb/src/execution/window_executor.cpp index 9406200e0..d18166d02 100644 --- a/src/duckdb/src/execution/window_executor.cpp +++ b/src/duckdb/src/execution/window_executor.cpp @@ -255,6 +255,8 @@ static idx_t FindRangeBound(const WindowInputColumn &over, const idx_t order_beg return FindTypedRangeBound(over, order_begin, order_end, boundary, chunk_idx, prev); case PhysicalType::INT128: return FindTypedRangeBound(over, order_begin, order_end, boundary, chunk_idx, prev); + case PhysicalType::UINT128: + return FindTypedRangeBound(over, order_begin, order_end, boundary, chunk_idx, prev); case PhysicalType::FLOAT: return FindTypedRangeBound(over, order_begin, order_end, boundary, chunk_idx, prev); case PhysicalType::DOUBLE: @@ -850,6 +852,14 @@ bool WindowAggregateExecutor::IsConstantAggregate() { return true; } +bool WindowAggregateExecutor::IsDistinctAggregate() { + if (!wexpr.aggregate) { + return false; + } + + return wexpr.distinct; +} + bool WindowAggregateExecutor::IsCustomAggregate() { if (!wexpr.aggregate) { return false; @@ -878,18 +888,25 @@ WindowAggregateExecutor::WindowAggregateExecutor(BoundWindowExpression &wexpr, C const ValidityMask &order_mask, WindowAggregationMode mode) : WindowExecutor(wexpr, context, count, partition_mask, order_mask), mode(mode), filter_executor(context) { - // Check for constant aggregate - if (IsConstantAggregate()) { - aggregator = make_uniq(AggregateObject(wexpr), wexpr.return_type, partition_mask, - wexpr.exclude_clause, count); - } else if (IsCustomAggregate()) { + // Force naive for SEPARATE mode or for (currently!) unsupported functionality + const auto force_naive = + !ClientConfig::GetConfig(context).enable_optimizer || mode == WindowAggregationMode::SEPARATE; + AggregateObject aggr(wexpr); + if (force_naive || (wexpr.distinct && wexpr.exclude_clause != WindowExcludeMode::NO_OTHER)) { + aggregator = make_uniq(aggr, wexpr.return_type, wexpr.exclude_clause, count); + } else if (IsDistinctAggregate()) { + // build a merge sort tree + // see https://dl.acm.org/doi/pdf/10.1145/3514221.3526184 + aggregator = make_uniq(aggr, wexpr.return_type, wexpr.exclude_clause, count, context); + } else if (IsConstantAggregate()) { aggregator = - make_uniq(AggregateObject(wexpr), wexpr.return_type, wexpr.exclude_clause, count); - } else if (wexpr.aggregate) { + make_uniq(aggr, wexpr.return_type, partition_mask, wexpr.exclude_clause, count); + } else if (IsCustomAggregate()) { + aggregator = make_uniq(aggr, wexpr.return_type, wexpr.exclude_clause, count); + } else { // 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, mode, wexpr.exclude_clause, count); + aggregator = make_uniq(aggr, wexpr.return_type, mode, wexpr.exclude_clause, count); } // evaluate the FILTER clause and stuff it into a large mask for compactness and reuse diff --git a/src/duckdb/src/execution/window_segment_tree.cpp b/src/duckdb/src/execution/window_segment_tree.cpp index 956e6a7f1..95c0e5ab1 100644 --- a/src/duckdb/src/execution/window_segment_tree.cpp +++ b/src/duckdb/src/execution/window_segment_tree.cpp @@ -3,8 +3,11 @@ #include "duckdb/common/algorithm.hpp" #include "duckdb/common/helper.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/execution/merge_sort_tree.hpp" +#include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/execution/window_executor.hpp" +#include #include namespace duckdb { @@ -262,13 +265,7 @@ class WindowCustomAggregatorState : public WindowAggregatorState { SubFrames frames; }; -WindowCustomAggregatorState::WindowCustomAggregatorState(const AggregateObject &aggr, - const WindowExcludeMode exclude_mode) - : aggr(aggr), state(aggr.function.state_size()), 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()); - +static void InitSubFrames(SubFrames &frames, const WindowExcludeMode exclude_mode) { idx_t nframes = 0; switch (exclude_mode) { case WindowExcludeMode::NO_OTHER: @@ -285,6 +282,16 @@ WindowCustomAggregatorState::WindowCustomAggregatorState(const AggregateObject & frames.resize(nframes, {0, 0}); } +WindowCustomAggregatorState::WindowCustomAggregatorState(const AggregateObject &aggr, + const WindowExcludeMode exclude_mode) + : aggr(aggr), state(aggr.function.state_size()), 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()); + + InitSubFrames(frames, exclude_mode); +} + WindowCustomAggregatorState::~WindowCustomAggregatorState() { if (aggr.function.destructor) { AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); @@ -310,20 +317,14 @@ unique_ptr WindowCustomAggregator::GetLocalState() const return make_uniq(aggr, exclude_mode); } -void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, - idx_t count, idx_t row_idx) const { +template +static void EvaluateSubFrames(const DataChunk &bounds, const WindowExcludeMode exclude_mode, idx_t count, idx_t row_idx, + SubFrames &frames, OP operation) { 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]); - auto &lcstate = lstate.Cast(); - auto &frames = lcstate.frames; - const_data_ptr_t gstate_p = nullptr; - if (gstate) { - auto &gcstate = gstate->Cast(); - gstate_p = gcstate.state.data(); - } for (idx_t i = 0, cur_row = row_idx; i < count; ++i, ++cur_row) { idx_t nframes = 0; if (exclude_mode == WindowExcludeMode::NO_OTHER) { @@ -350,7 +351,7 @@ void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const DataC // this is not too hard to arrange, but it may be that some subframes are contiguous, // and some are empty. - // WindowSegmentTreePart::LEFT + // WindowExcludePart::LEFT auto begin = begins[i]; auto end = (exclude_mode == WindowExcludeMode::CURRENT_ROW) ? cur_row : peer_begin[i]; end = MaxValue(begin, end); @@ -362,17 +363,237 @@ void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const DataC frames[nframes++] = FrameBounds(cur_row, cur_row + 1); } - // WindowSegmentTreePart::RIGHT + // WindowExcludePart::RIGHT end = ends[i]; begin = (exclude_mode == WindowExcludeMode::CURRENT_ROW) ? (cur_row + 1) : peer_end[i]; begin = MinValue(begin, end); frames[nframes++] = FrameBounds(begin, end); } + operation(i); + } +} + +void WindowCustomAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, + idx_t count, idx_t row_idx) const { + auto &lcstate = lstate.Cast(); + auto &frames = lcstate.frames; + const_data_ptr_t gstate_p = nullptr; + if (gstate) { + auto &gcstate = gstate->Cast(); + gstate_p = gcstate.state.data(); + } + + EvaluateSubFrames(bounds, exclude_mode, count, row_idx, frames, [&](idx_t i) { // Extract the range AggregateInputData aggr_input_data(aggr.GetFunctionData(), lstate.allocator); aggr.function.window(aggr_input_data, *partition_input, gstate_p, lcstate.state.data(), frames, result, i); + }); +} + +//===--------------------------------------------------------------------===// +// WindowNaiveAggregator +//===--------------------------------------------------------------------===// +WindowNaiveAggregator::WindowNaiveAggregator(AggregateObject aggr, const LogicalType &result_type, + const WindowExcludeMode exclude_mode_p, idx_t partition_count) + : WindowAggregator(std::move(aggr), result_type, exclude_mode_p, partition_count) { +} + +WindowNaiveAggregator::~WindowNaiveAggregator() { +} + +class WindowNaiveState : public WindowAggregatorState { +public: + struct HashRow { + explicit HashRow(WindowNaiveState &state) : state(state) { + } + + size_t operator()(const idx_t &i) const { + return state.Hash(i); + } + + WindowNaiveState &state; + }; + + struct EqualRow { + explicit EqualRow(WindowNaiveState &state) : state(state) { + } + + bool operator()(const idx_t &lhs, const idx_t &rhs) const { + return state.KeyEqual(lhs, rhs); + } + + WindowNaiveState &state; + }; + + using RowSet = std::unordered_set; + + explicit WindowNaiveState(const WindowNaiveAggregator &gstate); + + void Evaluate(const DataChunk &bounds, Vector &result, idx_t count, idx_t row_idx); + +protected: + //! Flush the accumulated intermediate states into the result states + void FlushStates(); + + //! Hashes a value for the hash table + size_t Hash(idx_t rid); + //! Compares two values for the hash table + bool KeyEqual(const idx_t &lhs, const idx_t &rhs); + + //! The global state + const WindowNaiveAggregator &gstate; + //! Data pointer that contains a vector of states, used for row aggregation + vector state; + //! Reused result state container for the aggregate + Vector statef; + //! A vector of pointers to "state", used for buffering intermediate aggregates + Vector statep; + //! Input data chunk, used for leaf segment aggregation + DataChunk leaves; + //! The rows beging updated. + SelectionVector update_sel; + //! Count of buffered values + idx_t flush_count; + //! The frame boundaries, used for EXCLUDE + SubFrames frames; + //! The optional hash table used for DISTINCT + Vector hashes; + HashRow hash_row; + EqualRow equal_row; + RowSet row_set; +}; + +WindowNaiveState::WindowNaiveState(const WindowNaiveAggregator &gstate) + : gstate(gstate), state(gstate.state_size * STANDARD_VECTOR_SIZE), statef(LogicalType::POINTER), + statep((LogicalType::POINTER)), flush_count(0), hashes(LogicalType::HASH), hash_row(*this), equal_row(*this), + row_set(STANDARD_VECTOR_SIZE, hash_row, equal_row) { + InitSubFrames(frames, gstate.exclude_mode); + + auto &inputs = const_cast(gstate.GetInputs()); + if (inputs.ColumnCount() > 0) { + leaves.Initialize(Allocator::DefaultAllocator(), inputs.GetTypes()); } + + update_sel.Initialize(); + + // Build the finalise vector that just points to the result states + data_ptr_t state_ptr = state.data(); + D_ASSERT(statef.GetVectorType() == VectorType::FLAT_VECTOR); + statef.SetVectorType(VectorType::CONSTANT_VECTOR); + statef.Flatten(STANDARD_VECTOR_SIZE); + auto fdata = FlatVector::GetData(statef); + for (idx_t i = 0; i < STANDARD_VECTOR_SIZE; ++i) { + fdata[i] = state_ptr; + state_ptr += gstate.state_size; + } +} + +void WindowNaiveState::FlushStates() { + if (!flush_count) { + return; + } + + auto &inputs = const_cast(gstate.GetInputs()); + leaves.Reference(inputs); + leaves.Slice(update_sel, flush_count); + + auto &aggr = gstate.aggr; + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + aggr.function.update(leaves.data.data(), aggr_input_data, leaves.ColumnCount(), statep, flush_count); + + flush_count = 0; +} + +size_t WindowNaiveState::Hash(idx_t rid) { + auto &inputs = const_cast(gstate.GetInputs()); + leaves.Reference(inputs); + + sel_t s = rid; + SelectionVector sel(&s); + leaves.Slice(sel, 1); + leaves.Hash(hashes); + + return *FlatVector::GetData(hashes); +} + +bool WindowNaiveState::KeyEqual(const idx_t &lhs, const idx_t &rhs) { + auto &inputs = const_cast(gstate.GetInputs()); + + sel_t l = lhs; + SelectionVector lsel(&l); + + sel_t r = rhs; + SelectionVector rsel(&r); + + sel_t f = 0; + SelectionVector fsel(&f); + + for (auto &input : inputs.data) { + Vector left(input, lsel, 1); + Vector right(input, rsel, 1); + if (!VectorOperations::NotDistinctFrom(left, right, nullptr, 1, nullptr, &fsel)) { + return false; + } + } + + return true; +} + +void WindowNaiveState::Evaluate(const DataChunk &bounds, Vector &result, idx_t count, idx_t row_idx) { + auto &aggr = gstate.aggr; + auto &filter_mask = gstate.GetFilterMask(); + + auto fdata = FlatVector::GetData(statef); + auto pdata = FlatVector::GetData(statep); + + EvaluateSubFrames(bounds, gstate.exclude_mode, count, row_idx, frames, [&](idx_t rid) { + auto agg_state = fdata[rid]; + aggr.function.initialize(agg_state); + + // Just update the aggregate with the unfiltered input rows + row_set.clear(); + for (const auto &frame : frames) { + for (auto f = frame.start; f < frame.end; ++f) { + if (!filter_mask.RowIsValid(f)) { + continue; + } + + // Filter out duplicates + if (aggr.IsDistinct() && !row_set.insert(f).second) { + continue; + } + + pdata[flush_count] = agg_state; + update_sel[flush_count++] = f; + if (flush_count >= STANDARD_VECTOR_SIZE) { + FlushStates(); + } + } + } + }); + + // Flush the final states + FlushStates(); + + // Finalise the result aggregates and write to the result + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + aggr.function.finalize(statef, aggr_input_data, result, count, 0); + + // Destruct the result aggregates + if (aggr.function.destructor) { + aggr.function.destructor(statef, aggr_input_data, count); + } +} + +unique_ptr WindowNaiveAggregator::GetLocalState() const { + return make_uniq(*this); +} + +void WindowNaiveAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, + idx_t count, idx_t row_idx) const { + auto &ldstate = lstate.Cast(); + ldstate.Evaluate(bounds, result, count, row_idx); } //===--------------------------------------------------------------------===// @@ -418,6 +639,9 @@ WindowSegmentTree::~WindowSegmentTree() { class WindowSegmentTreePart { public: + //! Right side nodes need to be cached and processed in reverse order + using RightEntry = std::pair; + enum FramePart : uint8_t { FULL = 0, LEFT = 1, RIGHT = 2 }; WindowSegmentTreePart(ArenaAllocator &allocator, const AggregateObject &aggr, DataChunk &inputs, @@ -432,7 +656,7 @@ class WindowSegmentTreePart { 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 + //! Writes result and calls destructors void Finalize(Vector &result, idx_t count); void Combine(WindowSegmentTreePart &other, idx_t count); @@ -440,18 +664,29 @@ class WindowSegmentTreePart { 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); +protected: + //! Initialises the accumulation state vector (statef) + void Initialize(idx_t count); + //! Accumulate upper tree levels + void EvaluateUpperLevels(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, idx_t count, + idx_t row_idx, FramePart frame_part); + void EvaluateLeaves(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, idx_t count, + idx_t row_idx, FramePart frame_part, FramePart leaf_part); + public: //! Allocator for aggregates ArenaAllocator &allocator; //! The aggregate function const AggregateObject &aggr; - //! The aggregate function + //! Order insensitive aggregate (we can optimise internal combines) + const bool order_insensitive; + //! The partition arguments DataChunk &inputs; //! The filtered rows in inputs const ValidityMask &filter_mask; //! The size of a single aggregate state const idx_t state_size; - //! Data pointer that contains a single state, used for intermediate window segment aggregation + //! Data pointer that contains a vector of states, used for intermediate window segment aggregation vector state; //! Input data chunk, used for leaf segment aggregation DataChunk leaves; @@ -465,6 +700,8 @@ class WindowSegmentTreePart { Vector statef; //! Count of buffered values idx_t flush_count; + //! Cache of right side tree ranges for ordered aggregates + vector right_stack; }; class WindowSegmentTreeState : public WindowAggregatorState { @@ -487,9 +724,10 @@ class WindowSegmentTreeState : public WindowAggregatorState { 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) { + : allocator(allocator), aggr(aggr), + order_insensitive(aggr.function.order_dependent == AggregateOrderDependent::NOT_ORDER_DEPENDENT), 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(); @@ -679,18 +917,43 @@ void WindowSegmentTree::Evaluate(WindowAggregatorState &lstate, const DataChunk 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) { + D_ASSERT(aggr.function.combine && tree.UseCombineAPI()); + + Initialize(count); + + if (order_insensitive) { + // First pass: aggregate the segment tree nodes with sharing + EvaluateUpperLevels(tree, begins, ends, count, row_idx, frame_part); - const auto cant_combine = (!aggr.function.combine || !tree.UseCombineAPI()); + // Second pass: aggregate the ragged leaves + EvaluateLeaves(tree, begins, ends, count, row_idx, frame_part, FramePart::FULL); + } else { + // Evaluate leaves in order + EvaluateLeaves(tree, begins, ends, count, row_idx, frame_part, FramePart::LEFT); + EvaluateUpperLevels(tree, begins, ends, count, row_idx, frame_part); + EvaluateLeaves(tree, begins, ends, count, row_idx, frame_part, FramePart::RIGHT); + } +} + +void WindowSegmentTreePart::Initialize(idx_t count) { + auto fdata = FlatVector::GetData(statef); + for (idx_t rid = 0; rid < count; ++rid) { + auto state_ptr = fdata[rid]; + aggr.function.initialize(state_ptr); + } +} + +void WindowSegmentTreePart::EvaluateUpperLevels(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, + idx_t count, idx_t row_idx, FramePart frame_part) { 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 + const auto max_level = tree.levels_flat_start.size() + 1; + right_stack.resize(max_level, {0, 0}); + // Share adjacent identical states // We do this first because we want to share only tree aggregations idx_t prev_begin = 1; @@ -700,12 +963,6 @@ void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t data_ptr_t prev_state = nullptr; for (idx_t rid = 0, cur_row = row_idx; rid < count; ++rid, ++cur_row) { auto state_ptr = fdata[rid]; - aggr.function.initialize(state_ptr); - - if (cant_combine) { - // Make sure we initialise all states - continue; - } auto begin = begin_on_curr_row ? cur_row + 1 : begins[rid]; auto end = end_on_curr_row ? cur_row : ends[rid]; @@ -715,7 +972,8 @@ void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t // Skip level 0 idx_t l_idx = 0; - for (; l_idx < tree.levels_flat_start.size() + 1; l_idx++) { + idx_t right_max = 0; + for (; l_idx < max_level; 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) { @@ -728,7 +986,7 @@ void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t break; } - if (l_idx == 1) { + if (order_insensitive && l_idx == 1) { prev_state = state_ptr; prev_begin = begin; prev_end = end; @@ -750,17 +1008,52 @@ void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t idx_t group_end = parent_end * tree.TREE_FANOUT; if (end != group_end) { if (l_idx) { - WindowSegmentValue(tree, l_idx, group_end, end, state_ptr); + if (order_insensitive) { + WindowSegmentValue(tree, l_idx, group_end, end, state_ptr); + } else { + right_stack[l_idx] = {group_end, end}; + right_max = l_idx; + } } } begin = parent_begin; end = parent_end; } + + // Flush the right side values from left to right for order_sensitive aggregates + // As we go up the tree, the right side ranges move left, + // so we just cache them in a fixed size, preallocated array. + // Then we can just reverse scan the array and append the cached ranges. + for (l_idx = right_max; l_idx > 0; --l_idx) { + auto &right_entry = right_stack[l_idx]; + const auto group_end = right_entry.first; + const auto end = right_entry.second; + if (end) { + WindowSegmentValue(tree, l_idx, group_end, end, state_ptr); + right_entry = {0, 0}; + } + } } FlushStates(true); +} + +void WindowSegmentTreePart::EvaluateLeaves(const WindowSegmentTree &tree, const idx_t *begins, const idx_t *ends, + idx_t count, idx_t row_idx, FramePart frame_part, FramePart leaf_part) { + + auto fdata = FlatVector::GetData(statef); + + // For order-sensitive aggregates, we have to process the ragged leaves in two pieces. + // The left side have to be added before the main tree followed by the ragged right sides. + // The current row is the leftmost value of the right hand side. + const bool compute_left = leaf_part != FramePart::RIGHT; + const bool compute_right = leaf_part != FramePart::LEFT; + 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 = compute_left && frame_part == FramePart::RIGHT && exclude_mode == WindowExcludeMode::TIES; - // Second pass: aggregate the ragged leaves - // (or everything if we can't combine) for (idx_t rid = 0, cur_row = row_idx; rid < count; ++rid, ++cur_row) { auto state_ptr = fdata[rid]; @@ -773,25 +1066,435 @@ void WindowSegmentTreePart::Evaluate(const WindowSegmentTree &tree, const idx_t continue; } - // Aggregate everything at once if we can't combine states idx_t parent_begin = begin / tree.TREE_FANOUT; idx_t parent_end = end / tree.TREE_FANOUT; - if (parent_begin == parent_end || cant_combine) { - WindowSegmentValue(tree, 0, begin, end, state_ptr); + if (parent_begin == parent_end) { + if (compute_left) { + WindowSegmentValue(tree, 0, begin, end, state_ptr); + } continue; } idx_t group_begin = parent_begin * tree.TREE_FANOUT; - if (begin != group_begin) { + if (begin != group_begin && compute_left) { WindowSegmentValue(tree, 0, begin, group_begin + tree.TREE_FANOUT, state_ptr); - parent_begin++; } idx_t group_end = parent_end * tree.TREE_FANOUT; - if (end != group_end) { + if (end != group_end && compute_right) { WindowSegmentValue(tree, 0, group_end, end, state_ptr); } } FlushStates(false); } +//===--------------------------------------------------------------------===// +// WindowDistinctAggregator +//===--------------------------------------------------------------------===// +WindowDistinctAggregator::WindowDistinctAggregator(AggregateObject aggr, const LogicalType &result_type, + const WindowExcludeMode exclude_mode_p, idx_t count, + ClientContext &context) + : WindowAggregator(std::move(aggr), result_type, exclude_mode_p, count), context(context), + allocator(Allocator::DefaultAllocator()) { + + payload_types.emplace_back(LogicalType::UBIGINT); + payload_chunk.Initialize(Allocator::DefaultAllocator(), payload_types); +} + +WindowDistinctAggregator::~WindowDistinctAggregator() { + if (!aggr.function.destructor) { + // nothing to destroy + return; + } + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + // call the destructor for all the intermediate states + data_ptr_t address_data[STANDARD_VECTOR_SIZE]; + Vector addresses(LogicalType::POINTER, data_ptr_cast(address_data)); + idx_t count = 0; + for (idx_t i = 0; i < internal_nodes; i++) { + address_data[count++] = data_ptr_t(levels_flat_native.get() + i * state_size); + if (count == STANDARD_VECTOR_SIZE) { + aggr.function.destructor(addresses, aggr_input_data, count); + count = 0; + } + } + if (count > 0) { + aggr.function.destructor(addresses, aggr_input_data, count); + } +} + +void WindowDistinctAggregator::Sink(DataChunk &arg_chunk, SelectionVector *filter_sel, idx_t filtered) { + WindowAggregator::Sink(arg_chunk, filter_sel, filtered); + + // We sort the arguments and use the partition index as a tie-breaker. + // TODO: Use a hash table? + if (!global_sort) { + // 1: functionComputePrevIdcs(𝑖𝑛) + // 2: sorted ← [] + vector sort_types; + for (const auto &col : arg_chunk.data) { + sort_types.emplace_back(col.GetType()); + } + + for (const auto &type : payload_types) { + sort_types.emplace_back(type); + } + + vector orders; + for (const auto &type : sort_types) { + auto expr = make_uniq(Value(type)); + orders.emplace_back(BoundOrderByNode(OrderType::ASCENDING, OrderByNullType::NULLS_FIRST, std::move(expr))); + } + + RowLayout payload_layout; + payload_layout.Initialize(payload_types); + + global_sort = make_uniq(BufferManager::GetBufferManager(context), orders, payload_layout); + local_sort.Initialize(*global_sort, global_sort->buffer_manager); + + sort_chunk.Initialize(Allocator::DefaultAllocator(), sort_types); + sort_chunk.data.back().Reference(payload_chunk.data[0]); + payload_pos = 0; + memory_per_thread = PhysicalOperator::GetMaxThreadMemory(context); + } + + // 3: for i ← 0 to in.size do + // 4: sorted[i] ← (in[i], i) + const auto count = arg_chunk.size(); + auto payload_data = FlatVector::GetData(payload_chunk.data[0]); + std::iota(payload_data, payload_data + count, payload_pos); + payload_pos += count; + + for (column_t c = 0; c < arg_chunk.ColumnCount(); ++c) { + sort_chunk.data[c].Reference(arg_chunk.data[c]); + } + sort_chunk.SetCardinality(arg_chunk); + payload_chunk.SetCardinality(sort_chunk); + + // Apply FILTER clause, if any + if (filter_sel) { + sort_chunk.Slice(*filter_sel, filtered); + payload_chunk.Slice(*filter_sel, filtered); + } + + local_sort.SinkChunk(sort_chunk, payload_chunk); + + if (local_sort.SizeInBytes() > memory_per_thread) { + local_sort.Sort(*global_sort, true); + } +} + +class WindowDistinctAggregator::DistinctSortTree : public MergeSortTree { +public: + // prev_idx, input_idx + using ZippedTuple = std::tuple; + using ZippedElements = vector; + + DistinctSortTree(ZippedElements &&prev_idcs, WindowDistinctAggregator &wda); +}; + +void WindowDistinctAggregator::Finalize(const FrameStats &stats) { + // 5: Sort sorted lexicographically increasing + global_sort->AddLocalState(local_sort); + global_sort->PrepareMergePhase(); + while (global_sort->sorted_blocks.size() > 1) { + global_sort->InitializeMergeRound(); + MergeSorter merge_sorter(*global_sort, global_sort->buffer_manager); + merge_sorter.PerformInMergeRound(); + global_sort->CompleteMergeRound(true); + } + + DataChunk scan_chunk; + scan_chunk.Initialize(Allocator::DefaultAllocator(), payload_types); + + auto scanner = make_uniq(*global_sort); + const auto in_size = scanner->Remaining(); + scanner->Scan(scan_chunk); + idx_t scan_idx = 0; + + // 6: prevIdcs ← [] + // 7: prevIdcs[0] ← “-” + const auto count = inputs.size(); + using ZippedTuple = DistinctSortTree::ZippedTuple; + DistinctSortTree::ZippedElements prev_idcs; + prev_idcs.resize(count); + + // To handle FILTER clauses we make the missing elements + // point to themselves so they won't be counted. + if (in_size < count) { + for (idx_t i = 0; i < count; ++i) { + prev_idcs[i] = ZippedTuple(i + 1, i); + } + } + + auto *input_idx = FlatVector::GetData(scan_chunk.data[0]); + auto i = input_idx[scan_idx++]; + prev_idcs[i] = ZippedTuple(0, i); + + SBIterator curr(*global_sort, ExpressionType::COMPARE_LESSTHAN); + SBIterator prev(*global_sort, ExpressionType::COMPARE_LESSTHAN); + auto prefix_layout = global_sort->sort_layout.GetPrefixComparisonLayout(sort_chunk.ColumnCount() - 1); + + // 8: for i ← 1 to in.size do + for (++curr; curr.GetIndex() < in_size; ++curr, ++prev) { + // Scan second one chunk at a time + // Note the scan is one behind the iterators + if (scan_idx >= scan_chunk.size()) { + scan_chunk.Reset(); + scanner->Scan(scan_chunk); + scan_idx = 0; + input_idx = FlatVector::GetData(scan_chunk.data[0]); + } + auto second = i; + i = input_idx[scan_idx++]; + + int lt = 0; + if (prefix_layout.all_constant) { + lt = FastMemcmp(prev.entry_ptr, curr.entry_ptr, prefix_layout.comparison_size); + } else { + lt = Comparators::CompareTuple(prev.scan, curr.scan, prev.entry_ptr, curr.entry_ptr, prefix_layout, + prev.external); + } + + // 9: if sorted[i].first == sorted[i-1].first then + // 10: prevIdcs[i] ← sorted[i-1].second + // 11: else + // 12: prevIdcs[i] ← “-” + if (!lt) { + prev_idcs[i] = ZippedTuple(second + 1, i); + } else { + prev_idcs[i] = ZippedTuple(0, i); + } + } + // 13: return prevIdcs + + merge_sort_tree = make_uniq(std::move(prev_idcs), *this); +} + +WindowDistinctAggregator::DistinctSortTree::DistinctSortTree(ZippedElements &&prev_idcs, + WindowDistinctAggregator &wda) { + auto &inputs = wda.inputs; + auto &aggr = wda.aggr; + auto &allocator = wda.allocator; + const auto state_size = wda.state_size; + auto &internal_nodes = wda.internal_nodes; + auto &levels_flat_native = wda.levels_flat_native; + auto &levels_flat_start = wda.levels_flat_start; + + //! Input data chunk, used for leaf segment aggregation + DataChunk leaves; + leaves.Initialize(Allocator::DefaultAllocator(), inputs.GetTypes()); + SelectionVector sel; + sel.Initialize(); + + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + + //! The states to update + Vector update_v(LogicalType::POINTER); + auto updates = FlatVector::GetData(update_v); + idx_t nupdate = 0; + + Vector source_v(LogicalType::POINTER); + auto sources = FlatVector::GetData(source_v); + Vector target_v(LogicalType::POINTER); + auto targets = FlatVector::GetData(target_v); + idx_t ncombine = 0; + + // compute space required to store aggregation states of merge sort tree + // this is one aggregate state per entry per level + MergeSortTree zipped_tree(std::move(prev_idcs)); + internal_nodes = 0; + for (idx_t level_nr = 0; level_nr < zipped_tree.tree.size(); ++level_nr) { + internal_nodes += zipped_tree.tree[level_nr].first.size(); + } + levels_flat_native = make_unsafe_uniq_array(internal_nodes * state_size); + levels_flat_start.push_back(0); + idx_t levels_flat_offset = 0; + + // Walk the distinct value tree building the intermediate aggregates + tree.reserve(zipped_tree.tree.size()); + idx_t level_width = 1; + for (idx_t level_nr = 0; level_nr < zipped_tree.tree.size(); ++level_nr) { + auto &zipped_level = zipped_tree.tree[level_nr].first; + vector level; + level.reserve(zipped_level.size()); + + for (idx_t i = 0; i < zipped_level.size(); i += level_width) { + // Reset the combine state + data_ptr_t prev_state = nullptr; + auto next_limit = MinValue(zipped_level.size(), i + level_width); + for (auto j = i; j < next_limit; ++j) { + // Initialise the next aggregate + auto curr_state = levels_flat_native.get() + (levels_flat_offset++ * state_size); + aggr.function.initialize(curr_state); + + // Update this state (if it matches) + const auto prev_idx = std::get<0>(zipped_level[j]); + level.emplace_back(prev_idx); + if (prev_idx < i + 1) { + updates[nupdate] = curr_state; + // input_idx + sel[nupdate] = std::get<1>(zipped_level[j]); + ++nupdate; + } + + // Merge the previous state (if any) + if (prev_state) { + sources[ncombine] = prev_state; + targets[ncombine] = curr_state; + ++ncombine; + } + prev_state = curr_state; + + // Flush the states if one is maxed out. + if (MaxValue(ncombine, nupdate) >= STANDARD_VECTOR_SIZE) { + // Push the updates first so they propagate + leaves.Reference(inputs); + leaves.Slice(sel, nupdate); + aggr.function.update(leaves.data.data(), aggr_input_data, leaves.ColumnCount(), update_v, nupdate); + nupdate = 0; + + // Combine the states sequentially + aggr.function.combine(source_v, target_v, aggr_input_data, ncombine); + ncombine = 0; + } + } + } + + tree.emplace_back(std::move(level), std::move(zipped_tree.tree[level_nr].second)); + + levels_flat_start.push_back(levels_flat_offset); + level_width *= FANOUT; + } + + // Flush any remaining states + if (ncombine || nupdate) { + // Push the updates + leaves.Reference(inputs); + leaves.Slice(sel, nupdate); + aggr.function.update(leaves.data.data(), aggr_input_data, leaves.ColumnCount(), update_v, nupdate); + nupdate = 0; + + // Combine the states sequentially + aggr.function.combine(source_v, target_v, aggr_input_data, ncombine); + ncombine = 0; + } +} + +class WindowDistinctState : public WindowAggregatorState { +public: + WindowDistinctState(const AggregateObject &aggr, DataChunk &inputs, const WindowDistinctAggregator &tree); + + void Evaluate(const DataChunk &bounds, Vector &result, idx_t count, idx_t row_idx); + +protected: + //! Flush the accumulated intermediate states into the result states + void FlushStates(); + + //! The aggregate function + const AggregateObject &aggr; + //! The aggregate function + DataChunk &inputs; + //! The merge sort tree data + const WindowDistinctAggregator &tree; + //! The size of a single aggregate state + const idx_t state_size; + //! Data pointer that contains a vector of states, used for row aggregation + vector state; + //! Reused result state container for the window functions + Vector statef; + //! A vector of pointers to "state", used for buffering intermediate aggregates + Vector statep; + //! Reused state pointers for combining tree elements + Vector statel; + //! Count of buffered values + idx_t flush_count; + //! The frame boundaries, used for the window functions + SubFrames frames; +}; + +WindowDistinctState::WindowDistinctState(const AggregateObject &aggr, DataChunk &inputs, + const WindowDistinctAggregator &tree) + : aggr(aggr), inputs(inputs), tree(tree), state_size(aggr.function.state_size()), + state((state_size * STANDARD_VECTOR_SIZE)), statef(LogicalType::POINTER), statep(LogicalType::POINTER), + statel(LogicalType::POINTER), flush_count(0) { + InitSubFrames(frames, tree.exclude_mode); + + // Build the finalise vector that just points to the result states + data_ptr_t state_ptr = state.data(); + D_ASSERT(statef.GetVectorType() == VectorType::FLAT_VECTOR); + statef.SetVectorType(VectorType::CONSTANT_VECTOR); + statef.Flatten(STANDARD_VECTOR_SIZE); + auto fdata = FlatVector::GetData(statef); + for (idx_t i = 0; i < STANDARD_VECTOR_SIZE; ++i) { + fdata[i] = state_ptr; + state_ptr += state_size; + } +} + +void WindowDistinctState::FlushStates() { + if (!flush_count) { + return; + } + + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + statel.Verify(flush_count); + aggr.function.combine(statel, statep, aggr_input_data, flush_count); + + flush_count = 0; +} + +void WindowDistinctState::Evaluate(const DataChunk &bounds, Vector &result, idx_t count, idx_t row_idx) { + auto fdata = FlatVector::GetData(statef); + auto ldata = FlatVector::GetData(statel); + auto pdata = FlatVector::GetData(statep); + + const auto &merge_sort_tree = *tree.merge_sort_tree; + const auto running_aggs = tree.levels_flat_native.get(); + + EvaluateSubFrames(bounds, tree.exclude_mode, count, row_idx, frames, [&](idx_t rid) { + auto agg_state = fdata[rid]; + aggr.function.initialize(agg_state); + + // TODO: Extend AggregateLowerBound to handle subframes, just like SelectNth. + const auto lower = frames[0].start; + const auto upper = frames[0].end; + merge_sort_tree.AggregateLowerBound(lower, upper, lower + 1, + [&](idx_t level, const idx_t run_begin, const idx_t run_pos) { + if (run_pos != run_begin) { + // Find the source aggregate + // Buffer a merge of the indicated state into the current state + const auto agg_idx = tree.levels_flat_start[level] + run_pos - 1; + const auto running_agg = running_aggs + agg_idx * state_size; + pdata[flush_count] = agg_state; + ldata[flush_count++] = running_agg; + if (flush_count >= STANDARD_VECTOR_SIZE) { + FlushStates(); + } + } + }); + }); + + // Flush the final states + FlushStates(); + + // Finalise the result aggregates and write to the result + AggregateInputData aggr_input_data(aggr.GetFunctionData(), allocator); + aggr.function.finalize(statef, aggr_input_data, result, count, 0); + + // Destruct the result aggregates + if (aggr.function.destructor) { + aggr.function.destructor(statef, aggr_input_data, count); + } +} + +unique_ptr WindowDistinctAggregator::GetLocalState() const { + return make_uniq(aggr, const_cast(inputs), *this); +} + +void WindowDistinctAggregator::Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, + idx_t count, idx_t row_idx) const { + auto &ldstate = lstate.Cast(); + ldstate.Evaluate(bounds, result, count, row_idx); +} + } // namespace duckdb diff --git a/src/duckdb/src/function/aggregate/distributive/count.cpp b/src/duckdb/src/function/aggregate/distributive/count.cpp index 99f020178..10ff1c46d 100644 --- a/src/duckdb/src/function/aggregate/distributive/count.cpp +++ b/src/duckdb/src/function/aggregate/distributive/count.cpp @@ -225,6 +225,7 @@ AggregateFunction CountStarFun::GetFunction() { auto fun = AggregateFunction::NullaryAggregate(LogicalType::BIGINT); fun.name = "count_star"; fun.null_handling = FunctionNullHandling::SPECIAL_HANDLING; + fun.order_dependent = AggregateOrderDependent::NOT_ORDER_DEPENDENT; fun.window = CountStarFunction::Window; return fun; } @@ -246,9 +247,7 @@ void CountFun::RegisterFunction(BuiltinFunctions &set) { AggregateFunctionSet count("count"); count.AddFunction(count_function); // the count function can also be called without arguments - count_function.arguments.clear(); - count_function.statistics = nullptr; - count_function.window = CountStarFunction::Window; + count_function = CountStarFun::GetFunction(); count.AddFunction(count_function); set.AddFunction(count); } diff --git a/src/duckdb/src/function/aggregate/distributive/first.cpp b/src/duckdb/src/function/aggregate/distributive/first.cpp index 8f8df8a70..65a50647c 100644 --- a/src/duckdb/src/function/aggregate/distributive/first.cpp +++ b/src/duckdb/src/function/aggregate/distributive/first.cpp @@ -1,6 +1,6 @@ -#include "duckdb/function/aggregate/distributive_functions.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" +#include "duckdb/function/aggregate/distributive_functions.hpp" #include "duckdb/planner/expression.hpp" namespace duckdb { @@ -85,7 +85,7 @@ struct FirstFunctionString : public FirstFunctionBase { } else { // non-inlined string, need to allocate space for it auto len = value.GetSize(); - auto ptr = new char[len]; + auto ptr = LAST ? new char[len] : char_ptr_cast(input_data.allocator.Allocate(len)); memcpy(ptr, value.GetData(), len); state.value = string_t(ptr, len); @@ -258,6 +258,8 @@ static AggregateFunction GetFirstFunction(const LogicalType &type) { return GetFirstAggregateTemplated(type); case LogicalTypeId::HUGEINT: return GetFirstAggregateTemplated(type); + case LogicalTypeId::UHUGEINT: + return GetFirstAggregateTemplated(type); case LogicalTypeId::FLOAT: return GetFirstAggregateTemplated(type); case LogicalTypeId::DOUBLE: @@ -266,8 +268,13 @@ static AggregateFunction GetFirstFunction(const LogicalType &type) { return GetFirstAggregateTemplated(type); case LogicalTypeId::VARCHAR: case LogicalTypeId::BLOB: - return AggregateFunction::UnaryAggregateDestructor, string_t, string_t, - FirstFunctionString>(type, type); + if (LAST) { + return AggregateFunction::UnaryAggregateDestructor, string_t, string_t, + FirstFunctionString>(type, type); + } else { + return AggregateFunction::UnaryAggregate, string_t, string_t, + FirstFunctionString>(type, type); + } case LogicalTypeId::DECIMAL: { type.Verify(); AggregateFunction function = GetDecimalFirstFunction(type); diff --git a/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp b/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp index a0c4951d5..054251f33 100644 --- a/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp +++ b/src/duckdb/src/function/aggregate/sorted_aggregate_function.cpp @@ -1,5 +1,6 @@ #include "duckdb/common/sort/sort.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" +#include "duckdb/common/types/list_segment.hpp" #include "duckdb/function/aggregate_function.hpp" #include "duckdb/function/function_binder.hpp" #include "duckdb/storage/buffer_manager.hpp" @@ -17,14 +18,22 @@ struct SortedAggregateBindData : public FunctionData { external(ClientConfig::GetConfig(context).force_external) { auto &children = expr.children; arg_types.reserve(children.size()); + arg_funcs.reserve(children.size()); for (const auto &child : children) { arg_types.emplace_back(child->return_type); + ListSegmentFunctions funcs; + GetSegmentDataFunctions(funcs, arg_types.back()); + arg_funcs.emplace_back(funcs); } auto &order_bys = *expr.order_bys; sort_types.reserve(order_bys.orders.size()); + sort_funcs.reserve(order_bys.orders.size()); for (auto &order : order_bys.orders) { orders.emplace_back(order.Copy()); sort_types.emplace_back(order.expression->return_type); + ListSegmentFunctions funcs; + GetSegmentDataFunctions(funcs, sort_types.back()); + sort_funcs.emplace_back(funcs); } sorted_on_args = (children.size() == order_bys.orders.size()); for (size_t i = 0; sorted_on_args && i < children.size(); ++i) { @@ -34,8 +43,8 @@ struct SortedAggregateBindData : public FunctionData { SortedAggregateBindData(const SortedAggregateBindData &other) : buffer_manager(other.buffer_manager), function(other.function), arg_types(other.arg_types), - sort_types(other.sort_types), sorted_on_args(other.sorted_on_args), threshold(other.threshold), - external(other.external) { + arg_funcs(other.arg_funcs), sort_types(other.sort_types), sort_funcs(other.sort_funcs), + sorted_on_args(other.sorted_on_args), threshold(other.threshold), external(other.external) { if (other.bind_info) { bind_info = other.bind_info->Copy(); } @@ -75,9 +84,11 @@ struct SortedAggregateBindData : public FunctionData { AggregateFunction function; vector arg_types; unique_ptr bind_info; + vector arg_funcs; vector orders; vector sort_types; + vector sort_funcs; bool sorted_on_args; //! The sort flush threshold @@ -86,125 +97,279 @@ struct SortedAggregateBindData : public FunctionData { }; struct SortedAggregateState { - //! Default buffer size, optimised for small group to avoid blowing out memory. - static const idx_t BUFFER_CAPACITY = 16; + // Linked list equivalent of DataChunk + using LinkedLists = vector; + using LinkedChunkFunctions = vector; + + //! Capacities of the various levels of buffering + static const idx_t LIST_CAPACITY = 16; + static const idx_t CHUNK_CAPACITY = STANDARD_VECTOR_SIZE; SortedAggregateState() : count(0), nsel(0), offset(0) { } - static inline void InitializeBuffer(DataChunk &chunk, const vector &types) { - if (!chunk.ColumnCount() && !types.empty()) { - chunk.Initialize(Allocator::DefaultAllocator(), types, BUFFER_CAPACITY); + static inline void InitializeLinkedList(LinkedLists &linked, const vector &types) { + if (linked.empty() && !types.empty()) { + linked.resize(types.size(), LinkedList()); } } - //! Make sure the buffer is large enough for slicing - static inline void ResetBuffer(DataChunk &chunk, const vector &types) { - chunk.Reset(); - chunk.Destroy(); - chunk.Initialize(Allocator::DefaultAllocator(), types); + inline void InitializeLinkedLists(const SortedAggregateBindData &order_bind) { + InitializeLinkedList(sort_linked, order_bind.sort_types); + if (!order_bind.sorted_on_args) { + InitializeLinkedList(arg_linked, order_bind.arg_types); + } } - void Flush(const SortedAggregateBindData &order_bind) { - if (ordering) { - return; + static inline void InitializeChunk(unique_ptr &chunk, const vector &types) { + if (!chunk && !types.empty()) { + chunk = make_uniq(); + chunk->Initialize(Allocator::DefaultAllocator(), types); + } + } + + void InitializeChunks(const SortedAggregateBindData &order_bind) { + // Lazy instantiation of the buffer chunks + InitializeChunk(sort_chunk, order_bind.sort_types); + if (!order_bind.sorted_on_args) { + InitializeChunk(arg_chunk, order_bind.arg_types); + } + } + + static inline void FlushLinkedList(const LinkedChunkFunctions &funcs, LinkedLists &linked, DataChunk &chunk) { + idx_t total_count = 0; + for (column_t i = 0; i < linked.size(); ++i) { + funcs[i].BuildListVector(linked[i], chunk.data[i], total_count); + chunk.SetCardinality(linked[i].total_capacity); + } + } + + void FlushLinkedLists(const SortedAggregateBindData &order_bind) { + InitializeChunks(order_bind); + FlushLinkedList(order_bind.sort_funcs, sort_linked, *sort_chunk); + if (arg_chunk) { + FlushLinkedList(order_bind.arg_funcs, arg_linked, *arg_chunk); } + } + void InitializeCollections(const SortedAggregateBindData &order_bind) { ordering = make_uniq(order_bind.buffer_manager, order_bind.sort_types); - InitializeBuffer(sort_buffer, order_bind.sort_types); - ordering->Append(sort_buffer); - ResetBuffer(sort_buffer, order_bind.sort_types); + ordering_append = make_uniq(); + ordering->InitializeAppend(*ordering_append); if (!order_bind.sorted_on_args) { arguments = make_uniq(order_bind.buffer_manager, order_bind.arg_types); - InitializeBuffer(arg_buffer, order_bind.arg_types); - arguments->Append(arg_buffer); - ResetBuffer(arg_buffer, order_bind.arg_types); + arguments_append = make_uniq(); + arguments->InitializeAppend(*arguments_append); } } - void Update(const SortedAggregateBindData &order_bind, DataChunk &sort_chunk, DataChunk &arg_chunk) { - count += sort_chunk.size(); + void FlushChunks(const SortedAggregateBindData &order_bind) { + D_ASSERT(sort_chunk); + ordering->Append(*ordering_append, *sort_chunk); + sort_chunk->Reset(); - // Lazy instantiation of the buffer chunks - InitializeBuffer(sort_buffer, order_bind.sort_types); - if (!order_bind.sorted_on_args) { - InitializeBuffer(arg_buffer, order_bind.arg_types); + if (arguments) { + D_ASSERT(arg_chunk); + arguments->Append(*arguments_append, *arg_chunk); + arg_chunk->Reset(); } + } - if (sort_chunk.size() + sort_buffer.size() > STANDARD_VECTOR_SIZE) { - Flush(order_bind); + void Resize(const SortedAggregateBindData &order_bind, idx_t n) { + count = n; + + // Establish the current buffering + if (count <= LIST_CAPACITY) { + InitializeLinkedLists(order_bind); } - if (arguments) { - ordering->Append(sort_chunk); - arguments->Append(arg_chunk); - } else if (ordering) { - ordering->Append(sort_chunk); - } else if (order_bind.sorted_on_args) { - sort_buffer.Append(sort_chunk, true); - } else { - sort_buffer.Append(sort_chunk, true); - arg_buffer.Append(arg_chunk, true); + + if (count > LIST_CAPACITY && !sort_chunk && !ordering) { + FlushLinkedLists(order_bind); + } + + if (count > CHUNK_CAPACITY && !ordering) { + InitializeCollections(order_bind); + FlushChunks(order_bind); + } + } + + static void LinkedAppend(const LinkedChunkFunctions &functions, ArenaAllocator &allocator, DataChunk &input, + LinkedLists &linked, SelectionVector &sel, idx_t nsel) { + const auto count = input.size(); + for (column_t c = 0; c < input.ColumnCount(); ++c) { + auto &func = functions[c]; + auto &linked_list = linked[c]; + RecursiveUnifiedVectorFormat input_data; + Vector::RecursiveToUnifiedFormat(input.data[c], count, input_data); + for (idx_t i = 0; i < nsel; ++i) { + idx_t sidx = sel.get_index(i); + func.AppendRow(allocator, linked_list, input_data, sidx); + } } } - void UpdateSlice(const SortedAggregateBindData &order_bind, DataChunk &sort_inputs, DataChunk &arg_inputs) { - count += nsel; + static void LinkedAbsorb(LinkedLists &source, LinkedLists &target) { + D_ASSERT(source.size() == target.size()); + for (column_t i = 0; i < source.size(); ++i) { + auto &src = source[i]; + if (!src.total_capacity) { + break; + } - // Lazy instantiation of the buffer chunks - InitializeBuffer(sort_buffer, order_bind.sort_types); - if (!order_bind.sorted_on_args) { - InitializeBuffer(arg_buffer, order_bind.arg_types); + auto &tgt = target[i]; + if (!tgt.total_capacity) { + tgt = src; + } else { + // append the linked list + tgt.last_segment->next = src.first_segment; + tgt.last_segment = src.last_segment; + tgt.total_capacity += src.total_capacity; + } } + } + + void Update(const AggregateInputData &aggr_input_data, DataChunk &sort_input, DataChunk &arg_input) { + const auto &order_bind = aggr_input_data.bind_data->Cast(); + Resize(order_bind, count + sort_input.size()); + + sel.Initialize(nullptr); + nsel = sort_input.size(); - if (nsel + sort_buffer.size() > STANDARD_VECTOR_SIZE) { - Flush(order_bind); + if (ordering) { + // Using collections + ordering->Append(*ordering_append, sort_input); + if (arguments) { + arguments->Append(*arguments_append, arg_input); + } + } else if (sort_chunk) { + // Still using data chunks + sort_chunk->Append(sort_input); + if (arg_chunk) { + arg_chunk->Append(arg_input); + } + } else { + // Still using linked lists + LinkedAppend(order_bind.sort_funcs, aggr_input_data.allocator, sort_input, sort_linked, sel, nsel); + if (!arg_linked.empty()) { + LinkedAppend(order_bind.arg_funcs, aggr_input_data.allocator, arg_input, arg_linked, sel, nsel); + } } - if (arguments) { - sort_buffer.Reset(); - sort_buffer.Slice(sort_inputs, sel, nsel); - ordering->Append(sort_buffer); - arg_buffer.Reset(); - arg_buffer.Slice(arg_inputs, sel, nsel); - arguments->Append(arg_buffer); - } else if (ordering) { - sort_buffer.Reset(); - sort_buffer.Slice(sort_inputs, sel, nsel); - ordering->Append(sort_buffer); - } else if (order_bind.sorted_on_args) { - sort_buffer.Append(sort_inputs, true, &sel, nsel); + nsel = 0; + offset = 0; + } + + void UpdateSlice(const AggregateInputData &aggr_input_data, DataChunk &sort_input, DataChunk &arg_input) { + const auto &order_bind = aggr_input_data.bind_data->Cast(); + Resize(order_bind, count + nsel); + + if (ordering) { + // Using collections + D_ASSERT(sort_chunk); + sort_chunk->Slice(sort_input, sel, nsel); + if (arg_chunk) { + arg_chunk->Slice(arg_input, sel, nsel); + } + FlushChunks(order_bind); + } else if (sort_chunk) { + // Still using data chunks + sort_chunk->Append(sort_input, true, &sel, nsel); + if (arg_chunk) { + arg_chunk->Append(arg_input, true, &sel, nsel); + } } else { - sort_buffer.Append(sort_inputs, true, &sel, nsel); - arg_buffer.Append(arg_inputs, true, &sel, nsel); + // Still using linked lists + LinkedAppend(order_bind.sort_funcs, aggr_input_data.allocator, sort_input, sort_linked, sel, nsel); + if (!arg_linked.empty()) { + LinkedAppend(order_bind.arg_funcs, aggr_input_data.allocator, arg_input, arg_linked, sel, nsel); + } } nsel = 0; offset = 0; } - void Combine(SortedAggregateBindData &order_bind, SortedAggregateState &other) { - if (other.arguments) { - // Force CDC if the other has it - Flush(order_bind); - ordering->Combine(*other.ordering); - arguments->Combine(*other.arguments); - count += other.count; - } else if (other.ordering) { - // Force CDC if the other has it - Flush(order_bind); - ordering->Combine(*other.ordering); - count += other.count; - } else if (other.sort_buffer.size()) { - Update(order_bind, other.sort_buffer, other.arg_buffer); + void Swap(SortedAggregateState &other) { + std::swap(count, other.count); + + std::swap(arguments, other.arguments); + std::swap(arguments_append, other.arguments_append); + std::swap(ordering, other.ordering); + std::swap(ordering_append, other.ordering_append); + + std::swap(sort_chunk, other.sort_chunk); + std::swap(arg_chunk, other.arg_chunk); + + std::swap(sort_linked, other.sort_linked); + std::swap(arg_linked, other.arg_linked); + } + + void Absorb(const SortedAggregateBindData &order_bind, SortedAggregateState &other) { + if (!other.count) { + return; + } else if (!count) { + Swap(other); + return; + } + + // Change to a state large enough for all the data + Resize(order_bind, count + other.count); + + // 3x3 matrix. + // We can simplify the logic a bit because the target is already set for the final capacity + if (!sort_chunk) { + // If the combined count is still linked lists, + // then just move the pointers. + // Note that this assumes ArenaAllocator is shared and the memory will not vanish under us. + LinkedAbsorb(other.sort_linked, sort_linked); + if (!arg_linked.empty()) { + LinkedAbsorb(other.arg_linked, arg_linked); + } + + other.Reset(); + return; + } + + if (!other.sort_chunk) { + other.FlushLinkedLists(order_bind); + } + + if (!ordering) { + // Still using chunks, which means the source is using chunks or lists + D_ASSERT(sort_chunk); + D_ASSERT(other.sort_chunk); + sort_chunk->Append(*other.sort_chunk); + if (arg_chunk) { + D_ASSERT(other.arg_chunk); + arg_chunk->Append(*other.arg_chunk); + } + } else { + // Using collections, so source could be using anything. + if (other.ordering) { + ordering->Combine(*other.ordering); + if (arguments) { + D_ASSERT(other.arguments); + arguments->Combine(*other.arguments); + } + } else { + ordering->Append(*other.sort_chunk); + if (arguments) { + D_ASSERT(other.arg_chunk); + arguments->Append(*other.arg_chunk); + } + } } + + // Free all memory as we have absorbed it. + other.Reset(); } void PrefixSortBuffer(DataChunk &prefixed) { - for (column_t col_idx = 0; col_idx < sort_buffer.ColumnCount(); ++col_idx) { - prefixed.data[col_idx + 1].Reference(sort_buffer.data[col_idx]); + for (column_t col_idx = 0; col_idx < sort_chunk->ColumnCount(); ++col_idx) { + prefixed.data[col_idx + 1].Reference(sort_chunk->data[col_idx]); } - prefixed.SetCardinality(sort_buffer); + prefixed.SetCardinality(*sort_chunk); } void Finalize(const SortedAggregateBindData &order_bind, DataChunk &prefixed, LocalSortState &local_sort) { @@ -213,37 +378,62 @@ struct SortedAggregateState { ordering->InitializeScan(sort_state); ColumnDataScanState arg_state; arguments->InitializeScan(arg_state); - for (sort_buffer.Reset(); ordering->Scan(sort_state, sort_buffer); sort_buffer.Reset()) { + for (sort_chunk->Reset(); ordering->Scan(sort_state, *sort_chunk); sort_chunk->Reset()) { PrefixSortBuffer(prefixed); - arg_buffer.Reset(); - arguments->Scan(arg_state, arg_buffer); - local_sort.SinkChunk(prefixed, arg_buffer); + arg_chunk->Reset(); + arguments->Scan(arg_state, *arg_chunk); + local_sort.SinkChunk(prefixed, *arg_chunk); } - ordering->Reset(); - arguments->Reset(); } else if (ordering) { ColumnDataScanState sort_state; ordering->InitializeScan(sort_state); - for (sort_buffer.Reset(); ordering->Scan(sort_state, sort_buffer); sort_buffer.Reset()) { + for (sort_chunk->Reset(); ordering->Scan(sort_state, *sort_chunk); sort_chunk->Reset()) { PrefixSortBuffer(prefixed); - local_sort.SinkChunk(prefixed, sort_buffer); + local_sort.SinkChunk(prefixed, *sort_chunk); } - ordering->Reset(); - } else if (order_bind.sorted_on_args) { - PrefixSortBuffer(prefixed); - local_sort.SinkChunk(prefixed, sort_buffer); } else { + // Force chunks so we can sort + if (!sort_chunk) { + FlushLinkedLists(order_bind); + } + PrefixSortBuffer(prefixed); - local_sort.SinkChunk(prefixed, arg_buffer); + if (arg_chunk) { + local_sort.SinkChunk(prefixed, *arg_chunk); + } else { + local_sort.SinkChunk(prefixed, *sort_chunk); + } } + + Reset(); + } + + void Reset() { + // Release all memory + ordering.reset(); + arguments.reset(); + + sort_chunk.reset(); + arg_chunk.reset(); + + sort_linked.clear(); + arg_linked.clear(); + + count = 0; } idx_t count; + unique_ptr arguments; + unique_ptr arguments_append; unique_ptr ordering; + unique_ptr ordering_append; + + unique_ptr sort_chunk; + unique_ptr arg_chunk; - DataChunk sort_buffer; - DataChunk arg_buffer; + LinkedLists sort_linked; + LinkedLists arg_linked; // Selection for scattering SelectionVector sel; @@ -263,33 +453,33 @@ struct SortedAggregateFunction { } static void ProjectInputs(Vector inputs[], const SortedAggregateBindData &order_bind, idx_t input_count, - idx_t count, DataChunk &arg_chunk, DataChunk &sort_chunk) { + idx_t count, DataChunk &arg_input, DataChunk &sort_input) { idx_t col = 0; if (!order_bind.sorted_on_args) { - arg_chunk.InitializeEmpty(order_bind.arg_types); - for (auto &dst : arg_chunk.data) { + arg_input.InitializeEmpty(order_bind.arg_types); + for (auto &dst : arg_input.data) { dst.Reference(inputs[col++]); } - arg_chunk.SetCardinality(count); + arg_input.SetCardinality(count); } - sort_chunk.InitializeEmpty(order_bind.sort_types); - for (auto &dst : sort_chunk.data) { + sort_input.InitializeEmpty(order_bind.sort_types); + for (auto &dst : sort_input.data) { dst.Reference(inputs[col++]); } - sort_chunk.SetCardinality(count); + sort_input.SetCardinality(count); } static void SimpleUpdate(Vector inputs[], AggregateInputData &aggr_input_data, idx_t input_count, data_ptr_t state, idx_t count) { const auto order_bind = aggr_input_data.bind_data->Cast(); - DataChunk arg_chunk; - DataChunk sort_chunk; - ProjectInputs(inputs, order_bind, input_count, count, arg_chunk, sort_chunk); + DataChunk arg_input; + DataChunk sort_input; + ProjectInputs(inputs, order_bind, input_count, count, arg_input, sort_input); const auto order_state = reinterpret_cast(state); - order_state->Update(order_bind, sort_chunk, arg_chunk); + order_state->Update(aggr_input_data, sort_input, arg_input); } static void ScatterUpdate(Vector inputs[], AggregateInputData &aggr_input_data, idx_t input_count, Vector &states, @@ -340,7 +530,7 @@ struct SortedAggregateFunction { continue; } - order_state->UpdateSlice(order_bind, sort_inputs, arg_inputs); + order_state->UpdateSlice(aggr_input_data, sort_inputs, arg_inputs); } } @@ -348,7 +538,7 @@ struct SortedAggregateFunction { static void Combine(const STATE &source, STATE &target, AggregateInputData &aggr_input_data) { auto &order_bind = aggr_input_data.bind_data->Cast(); auto &other = const_cast(source); - target.Combine(order_bind, other); + target.Absorb(order_bind, other); } static void Window(AggregateInputData &aggr_input_data, const WindowPartitionInput &partition, @@ -374,8 +564,7 @@ struct SortedAggregateFunction { // State variables auto bind_info = order_bind.bind_info.get(); - ArenaAllocator allocator(Allocator::DefaultAllocator()); - AggregateInputData aggr_bind_info(bind_info, allocator); + AggregateInputData aggr_bind_info(bind_info, aggr_input_data.allocator); // Inner aggregate APIs auto initialize = order_bind.function.initialize; @@ -525,29 +714,12 @@ void FunctionBinder::BindSortedAggregate(ClientContext &context, BoundAggregateE // not a sorted aggregate: return return; } + // Remove unnecessary ORDER BY clauses and return if nothing remains if (context.config.enable_optimizer) { - // for each ORDER BY - check if it is actually necessary - // expressions that are in the groups do not need to be ORDERED BY - // `ORDER BY` on a group has no effect, because for each aggregate, the group is unique - // similarly, we only need to ORDER BY each aggregate once - expression_set_t seen_expressions; - for (auto &target : groups) { - seen_expressions.insert(*target); - } - vector new_order_nodes; - for (auto &order_node : expr.order_bys->orders) { - if (seen_expressions.find(*order_node.expression) != seen_expressions.end()) { - // we do not need to order by this node - continue; - } - seen_expressions.insert(*order_node.expression); - new_order_nodes.push_back(std::move(order_node)); - } - if (new_order_nodes.empty()) { + if (expr.order_bys->Simplify(groups)) { expr.order_bys.reset(); return; } - expr.order_bys->orders = std::move(new_order_nodes); } auto &bound_function = expr.function; auto &children = expr.children; diff --git a/src/duckdb/src/function/cast/bit_cast.cpp b/src/duckdb/src/function/cast/bit_cast.cpp index aaffc12d1..6f01047f3 100644 --- a/src/duckdb/src/function/cast/bit_cast.cpp +++ b/src/duckdb/src/function/cast/bit_cast.cpp @@ -30,6 +30,8 @@ BoundCastInfo DefaultCasts::BitCastSwitch(BindCastInput &input, const LogicalTyp return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::HUGEINT: return BoundCastInfo(&VectorCastHelpers::TryCastLoop); + case LogicalTypeId::UHUGEINT: + return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::FLOAT: return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::DOUBLE: diff --git a/src/duckdb/src/function/cast/cast_function_set.cpp b/src/duckdb/src/function/cast/cast_function_set.cpp index 6e239f8fe..ce8c9943d 100644 --- a/src/duckdb/src/function/cast/cast_function_set.cpp +++ b/src/duckdb/src/function/cast/cast_function_set.cpp @@ -1,4 +1,3 @@ - #include "duckdb/function/cast/cast_function_set.hpp" #include "duckdb/common/pair.hpp" @@ -26,6 +25,10 @@ CastFunctionSet::CastFunctionSet() : map_info(nullptr) { bind_functions.emplace_back(DefaultCasts::GetDefaultCastFunction); } +CastFunctionSet::CastFunctionSet(DBConfig &config_p) : CastFunctionSet() { + this->config = &config_p; +} + CastFunctionSet &CastFunctionSet::Get(ClientContext &context) { return DBConfig::GetConfig(context).GetCastFunctions(); } @@ -156,7 +159,13 @@ int64_t CastFunctionSet::ImplicitCastCost(const LogicalType &source, const Logic } } // if not, fallback to the default implicit cast rules - return CastRules::ImplicitCast(source, target); + auto score = CastRules::ImplicitCast(source, target); + if (score < 0 && config && config->options.old_implicit_casting) { + if (source.id() != LogicalTypeId::BLOB && target.id() == LogicalTypeId::VARCHAR) { + score = 149; + } + } + return score; } BoundCastInfo MapCastFunction(BindCastInput &input, const LogicalType &source, const LogicalType &target) { diff --git a/src/duckdb/src/function/cast/decimal_cast.cpp b/src/duckdb/src/function/cast/decimal_cast.cpp index e10a77990..6f16bb426 100644 --- a/src/duckdb/src/function/cast/decimal_cast.cpp +++ b/src/duckdb/src/function/cast/decimal_cast.cpp @@ -247,6 +247,8 @@ BoundCastInfo DefaultCasts::DecimalCastSwitch(BindCastInput &input, const Logica return FromDecimalCast; case LogicalTypeId::HUGEINT: return FromDecimalCast; + case LogicalTypeId::UHUGEINT: + return FromDecimalCast; case LogicalTypeId::DECIMAL: { // decimal to decimal cast // first we need to figure out the source and target internal types diff --git a/src/duckdb/src/function/cast/default_casts.cpp b/src/duckdb/src/function/cast/default_casts.cpp index 60c5eb526..99b680456 100644 --- a/src/duckdb/src/function/cast/default_casts.cpp +++ b/src/duckdb/src/function/cast/default_casts.cpp @@ -5,7 +5,7 @@ #include "duckdb/common/operator/cast_operators.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/cast_helpers.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/function/cast/vector_cast_helpers.hpp" @@ -91,6 +91,7 @@ BoundCastInfo DefaultCasts::GetDefaultCastFunction(BindCastInput &input, const L case LogicalTypeId::USMALLINT: case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::HUGEINT: case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: diff --git a/src/duckdb/src/function/cast/list_casts.cpp b/src/duckdb/src/function/cast/list_casts.cpp index c0911b28d..13513e346 100644 --- a/src/duckdb/src/function/cast/list_casts.cpp +++ b/src/duckdb/src/function/cast/list_casts.cpp @@ -38,11 +38,14 @@ bool ListCast::ListToListCast(Vector &source, Vector &result, idx_t count, CastP // only handle constant and flat vectors here for now if (source.GetVectorType() == VectorType::CONSTANT_VECTOR) { result.SetVectorType(source.GetVectorType()); - ConstantVector::SetNull(result, ConstantVector::IsNull(source)); + const bool is_null = ConstantVector::IsNull(source); + ConstantVector::SetNull(result, is_null); - auto ldata = ConstantVector::GetData(source); - auto tdata = ConstantVector::GetData(result); - *tdata = *ldata; + if (!is_null) { + auto ldata = ConstantVector::GetData(source); + auto tdata = ConstantVector::GetData(result); + *tdata = *ldata; + } } else { source.Flatten(count); result.SetVectorType(VectorType::FLAT_VECTOR); diff --git a/src/duckdb/src/function/cast/numeric_casts.cpp b/src/duckdb/src/function/cast/numeric_casts.cpp index 071b0604b..0438e6e15 100644 --- a/src/duckdb/src/function/cast/numeric_casts.cpp +++ b/src/duckdb/src/function/cast/numeric_casts.cpp @@ -29,6 +29,8 @@ static BoundCastInfo InternalNumericCastSwitch(const LogicalType &source, const return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::HUGEINT: return BoundCastInfo(&VectorCastHelpers::TryCastLoop); + case LogicalTypeId::UHUGEINT: + return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::FLOAT: return BoundCastInfo(&VectorCastHelpers::TryCastLoop); case LogicalTypeId::DOUBLE: @@ -67,6 +69,8 @@ BoundCastInfo DefaultCasts::NumericCastSwitch(BindCastInput &input, const Logica return InternalNumericCastSwitch(source, target); case LogicalTypeId::HUGEINT: return InternalNumericCastSwitch(source, target); + case LogicalTypeId::UHUGEINT: + return InternalNumericCastSwitch(source, target); case LogicalTypeId::FLOAT: return InternalNumericCastSwitch(source, target); case LogicalTypeId::DOUBLE: diff --git a/src/duckdb/src/function/cast/string_cast.cpp b/src/duckdb/src/function/cast/string_cast.cpp index dc4f0c76f..7999e49e8 100644 --- a/src/duckdb/src/function/cast/string_cast.cpp +++ b/src/duckdb/src/function/cast/string_cast.cpp @@ -1,5 +1,6 @@ #include "duckdb/function/cast/default_casts.hpp" #include "duckdb/function/cast/vector_cast_helpers.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" #include "duckdb/common/pair.hpp" #include "duckdb/common/vector.hpp" #include "duckdb/function/scalar/nested_functions.hpp" @@ -102,6 +103,8 @@ static BoundCastInfo VectorStringCastNumericSwitch(BindCastInput &input, const L return BoundCastInfo(&VectorCastHelpers::TryCastStrictLoop); case LogicalTypeId::HUGEINT: return BoundCastInfo(&VectorCastHelpers::TryCastStrictLoop); + case LogicalTypeId::UHUGEINT: + return BoundCastInfo(&VectorCastHelpers::TryCastStrictLoop); case LogicalTypeId::FLOAT: return BoundCastInfo(&VectorCastHelpers::TryCastStrictLoop); case LogicalTypeId::DOUBLE: @@ -189,13 +192,16 @@ bool VectorStringToStruct::StringToNestedTypeCastLoop(const string_t *source_dat Vector varchar_vector(varchar_struct_type, count); auto &child_vectors = StructVector::GetEntries(varchar_vector); auto &result_children = StructVector::GetEntries(result); + auto is_unnamed = StructType::IsUnnamed(result.GetType()); string_map_t child_names; - vector child_masks; + vector> child_masks; for (idx_t child_idx = 0; child_idx < result_children.size(); child_idx++) { - child_names.insert({StructType::GetChildName(result.GetType(), child_idx), child_idx}); - child_masks.emplace_back(&FlatVector::Validity(*child_vectors[child_idx])); - child_masks[child_idx]->SetAllInvalid(count); + if (!is_unnamed) { + child_names.insert({StructType::GetChildName(result.GetType(), child_idx), child_idx}); + } + child_masks.emplace_back(FlatVector::Validity(*child_vectors[child_idx])); + child_masks[child_idx].get().SetAllInvalid(count); } bool all_converted = true; @@ -208,11 +214,14 @@ bool VectorStringToStruct::StringToNestedTypeCastLoop(const string_t *source_dat result_mask.SetInvalid(i); continue; } + if (is_unnamed) { + throw ConversionException("Casting strings to unnamed structs is unsupported"); + } if (!VectorStringToStruct::SplitStruct(source_data[idx], child_vectors, i, child_names, child_masks)) { string text = "Type VARCHAR with value '" + source_data[idx].GetString() + "' can't be cast to the destination type STRUCT"; for (auto &child_mask : child_masks) { - child_mask->SetInvalid(idx); // some values may have already been found and set valid + child_mask.get().SetInvalid(idx); // some values may have already been found and set valid } HandleVectorCastError::Operation(text, result_mask, idx, parameters.error_message, all_converted); } @@ -359,7 +368,7 @@ bool VectorStringToArray::StringToNestedTypeCastLoop(const string_t *source_data ", the size of the array must match the destination type", source_data[idx].GetString(), array_size); if (parameters.strict) { - throw CastException(msg); + throw ConversionException(msg); } HandleCastError::AssignError(msg, parameters.error_message); } diff --git a/src/duckdb/src/function/cast/struct_cast.cpp b/src/duckdb/src/function/cast/struct_cast.cpp index b40949acb..3fbb46b53 100644 --- a/src/duckdb/src/function/cast/struct_cast.cpp +++ b/src/duckdb/src/function/cast/struct_cast.cpp @@ -80,6 +80,7 @@ static bool StructToVarcharCast(Vector &source, Vector &result, idx_t count, Cas // now construct the actual varchar vector varchar_struct.Flatten(count); + bool is_unnamed = StructType::IsUnnamed(source.GetType()); auto &child_types = StructType::GetChildTypes(source.GetType()); auto &children = StructVector::GetEntries(varchar_struct); auto &validity = FlatVector::Validity(varchar_struct); @@ -101,13 +102,15 @@ static bool StructToVarcharCast(Vector &source, Vector &result, idx_t count, Cas auto &child_validity = FlatVector::Validity(*children[c]); auto data = FlatVector::GetData(*children[c]); auto &name = child_types[c].first; - string_length += name.size() + NAME_SEP_LENGTH; // "'{name}': " + if (!is_unnamed) { + string_length += name.size() + NAME_SEP_LENGTH; // "'{name}': " + } string_length += child_validity.RowIsValid(i) ? data[i].GetSize() : NULL_LENGTH; } result_data[i] = StringVector::EmptyString(result, string_length); auto dataptr = result_data[i].GetDataWriteable(); idx_t offset = 0; - dataptr[offset++] = '{'; + dataptr[offset++] = is_unnamed ? '(' : '{'; for (idx_t c = 0; c < children.size(); c++) { if (c > 0) { memcpy(dataptr + offset, ", ", SEP_LENGTH); @@ -115,14 +118,16 @@ static bool StructToVarcharCast(Vector &source, Vector &result, idx_t count, Cas } auto &child_validity = FlatVector::Validity(*children[c]); auto data = FlatVector::GetData(*children[c]); - auto &name = child_types[c].first; - // "'{name}': " - dataptr[offset++] = '\''; - memcpy(dataptr + offset, name.c_str(), name.size()); - offset += name.size(); - dataptr[offset++] = '\''; - dataptr[offset++] = ':'; - dataptr[offset++] = ' '; + if (!is_unnamed) { + auto &name = child_types[c].first; + // "'{name}': " + dataptr[offset++] = '\''; + memcpy(dataptr + offset, name.c_str(), name.size()); + offset += name.size(); + dataptr[offset++] = '\''; + dataptr[offset++] = ':'; + dataptr[offset++] = ' '; + } // value if (child_validity.RowIsValid(i)) { auto len = data[i].GetSize(); @@ -133,7 +138,7 @@ static bool StructToVarcharCast(Vector &source, Vector &result, idx_t count, Cas offset += NULL_LENGTH; } } - dataptr[offset++] = '}'; + dataptr[offset++] = is_unnamed ? ')' : '}'; result_data[i].Finalize(); } diff --git a/src/duckdb/src/function/cast/union/from_struct.cpp b/src/duckdb/src/function/cast/union/from_struct.cpp index 559803b86..ca3aecfca 100644 --- a/src/duckdb/src/function/cast/union/from_struct.cpp +++ b/src/duckdb/src/function/cast/union/from_struct.cpp @@ -1,4 +1,5 @@ #include "duckdb/function/cast/bound_cast_data.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/function/cast/union_casts.cpp b/src/duckdb/src/function/cast/union_casts.cpp index 280332458..1041958f5 100644 --- a/src/duckdb/src/function/cast/union_casts.cpp +++ b/src/duckdb/src/function/cast/union_casts.cpp @@ -1,6 +1,7 @@ #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/exception/conversion_exception.hpp" #include // for std::sort @@ -41,7 +42,7 @@ unique_ptr BindToUnionCast(BindCastInput &input, const LogicalTyp message += ", "; } } - throw CastException(message); + throw ConversionException(message); } // sort the candidate casts by cost @@ -68,7 +69,7 @@ unique_ptr BindToUnionCast(BindCastInput &input, const LogicalTyp } message += ". Disambiguate the target type by using the 'union_value( := )' function to promote the " "source value to a single member union before casting."; - throw CastException(message); + throw ConversionException(message); } // otherwise, return the selected cast @@ -182,7 +183,7 @@ unique_ptr BindUnionToUnionCast(BindCastInput &input, const Logic auto message = StringUtil::Format("Type %s can't be cast as %s. The member '%s' is not present in target union", source.ToString(), target.ToString(), source_member_name); - throw CastException(message); + throw ConversionException(message); } } diff --git a/src/duckdb/src/function/cast/vector_cast_helpers.cpp b/src/duckdb/src/function/cast/vector_cast_helpers.cpp index 98417a883..dde8f7701 100644 --- a/src/duckdb/src/function/cast/vector_cast_helpers.cpp +++ b/src/duckdb/src/function/cast/vector_cast_helpers.cpp @@ -285,7 +285,7 @@ static bool FindKeyStruct(const char *buf, idx_t len, idx_t &pos) { } static bool FindValueStruct(const char *buf, idx_t len, idx_t &pos, Vector &varchar_child, idx_t &row_idx, - ValidityMask *child_mask) { + ValidityMask &child_mask) { auto start_pos = pos; idx_t lvl = 0; while (pos < len) { @@ -302,7 +302,7 @@ static bool FindValueStruct(const char *buf, idx_t len, idx_t &pos, Vector &varc } FlatVector::GetData(varchar_child)[row_idx] = StringVector::AddString(varchar_child, buf + start_pos, end_pos - start_pos); - child_mask->SetValid(row_idx); // any child not set to valid will remain invalid + child_mask.SetValid(row_idx); // any child not set to valid will remain invalid return true; } pos++; @@ -312,7 +312,7 @@ static bool FindValueStruct(const char *buf, idx_t len, idx_t &pos, Vector &varc bool VectorStringToStruct::SplitStruct(const string_t &input, vector> &varchar_vectors, idx_t &row_idx, string_map_t &child_names, - vector &child_masks) { + vector> &child_masks) { const char *buf = input.GetData(); idx_t len = input.GetSize(); idx_t pos = 0; @@ -332,6 +332,10 @@ bool VectorStringToStruct::SplitStruct(const string_t &input, vector= key_end) { + // empty key name unsupported + return false; + } string_t found_key(buf + key_start, key_end - key_start); auto it = child_names.find(found_key); @@ -340,7 +344,7 @@ bool VectorStringToStruct::SplitStruct(const string_t &input, vectorsecond; SkipWhitespace(buf, ++pos, len); - if (!FindValueStruct(buf, len, pos, *varchar_vectors[child_idx], row_idx, child_masks[child_idx])) { + if (!FindValueStruct(buf, len, pos, *varchar_vectors[child_idx], row_idx, child_masks[child_idx].get())) { return false; } SkipWhitespace(buf, ++pos, len); diff --git a/src/duckdb/src/function/cast_rules.cpp b/src/duckdb/src/function/cast_rules.cpp index 239a9e22b..84a4bd03b 100644 --- a/src/duckdb/src/function/cast_rules.cpp +++ b/src/duckdb/src/function/cast_rules.cpp @@ -5,26 +5,34 @@ namespace duckdb { //! The target type determines the preferred implicit casts static int64_t TargetTypeCost(const LogicalType &type) { switch (type.id()) { - case LogicalTypeId::INTEGER: - return 103; case LogicalTypeId::BIGINT: return 101; - case LogicalTypeId::DOUBLE: + case LogicalTypeId::INTEGER: return 102; case LogicalTypeId::HUGEINT: - return 120; + return 103; + case LogicalTypeId::DOUBLE: + return 104; + case LogicalTypeId::DECIMAL: + return 105; + case LogicalTypeId::TIMESTAMP_NS: + return 119; case LogicalTypeId::TIMESTAMP: return 120; + case LogicalTypeId::TIMESTAMP_MS: + return 121; + case LogicalTypeId::TIMESTAMP_SEC: + return 122; case LogicalTypeId::VARCHAR: return 149; - case LogicalTypeId::DECIMAL: - return 104; case LogicalTypeId::STRUCT: case LogicalTypeId::MAP: case LogicalTypeId::LIST: case LogicalTypeId::UNION: case LogicalTypeId::ARRAY: return 160; + case LogicalTypeId::ANY: + return int64_t(AnyType::GetCastScore(type)); default: return 110; } @@ -93,6 +101,7 @@ static int64_t ImplicitCastUTinyint(const LogicalType &to) { case LogicalTypeId::INTEGER: case LogicalTypeId::BIGINT: case LogicalTypeId::HUGEINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: case LogicalTypeId::DECIMAL: @@ -109,6 +118,7 @@ static int64_t ImplicitCastUSmallint(const LogicalType &to) { case LogicalTypeId::INTEGER: case LogicalTypeId::BIGINT: case LogicalTypeId::HUGEINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: case LogicalTypeId::DECIMAL: @@ -123,6 +133,7 @@ static int64_t ImplicitCastUInteger(const LogicalType &to) { case LogicalTypeId::UBIGINT: case LogicalTypeId::BIGINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::HUGEINT: case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: @@ -137,6 +148,7 @@ static int64_t ImplicitCastUBigint(const LogicalType &to) { switch (to.id()) { case LogicalTypeId::FLOAT: case LogicalTypeId::DOUBLE: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::HUGEINT: case LogicalTypeId::DECIMAL: return TargetTypeCost(to); @@ -182,6 +194,17 @@ static int64_t ImplicitCastHugeint(const LogicalType &to) { } } +static int64_t ImplicitCastUhugeint(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::FLOAT: + case LogicalTypeId::DOUBLE: + case LogicalTypeId::DECIMAL: + return TargetTypeCost(to); + default: + return -1; + } +} + static int64_t ImplicitCastDate(const LogicalType &to) { switch (to.id()) { case LogicalTypeId::TIMESTAMP: @@ -191,8 +214,106 @@ static int64_t ImplicitCastDate(const LogicalType &to) { } } +static int64_t ImplicitCastEnum(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::VARCHAR: + return TargetTypeCost(to); + default: + return -1; + } +} + +static int64_t ImplicitCastTimestampSec(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::TIMESTAMP: + case LogicalTypeId::TIMESTAMP_MS: + case LogicalTypeId::TIMESTAMP_NS: + return TargetTypeCost(to); + default: + return -1; + } +} + +static int64_t ImplicitCastTimestampMS(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::TIMESTAMP: + case LogicalTypeId::TIMESTAMP_NS: + return TargetTypeCost(to); + default: + return -1; + } +} + +static int64_t ImplicitCastTimestampNS(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::TIMESTAMP: + // we allow casting ALL timestamps, including nanosecond ones, to TimestampNS + return TargetTypeCost(to); + default: + return -1; + } +} + +static int64_t ImplicitCastTimestamp(const LogicalType &to) { + switch (to.id()) { + case LogicalTypeId::TIMESTAMP_NS: + return TargetTypeCost(to); + default: + return -1; + } +} + +bool LogicalTypeIsValid(const LogicalType &type) { + switch (type.id()) { + case LogicalTypeId::STRUCT: + case LogicalTypeId::UNION: + case LogicalTypeId::LIST: + case LogicalTypeId::MAP: + case LogicalTypeId::ARRAY: + case LogicalTypeId::DECIMAL: + // these types are only valid with auxiliary info + if (!type.AuxInfo()) { + return false; + } + break; + default: + break; + } + switch (type.id()) { + case LogicalTypeId::ANY: + case LogicalTypeId::INVALID: + case LogicalTypeId::UNKNOWN: + return false; + case LogicalTypeId::STRUCT: { + auto child_count = StructType::GetChildCount(type); + for (idx_t i = 0; i < child_count; i++) { + if (!LogicalTypeIsValid(StructType::GetChildType(type, i))) { + return false; + } + } + return true; + } + case LogicalTypeId::UNION: { + auto member_count = UnionType::GetMemberCount(type); + for (idx_t i = 0; i < member_count; i++) { + if (!LogicalTypeIsValid(UnionType::GetMemberType(type, i))) { + return false; + } + } + return true; + } + case LogicalTypeId::LIST: + case LogicalTypeId::MAP: + return LogicalTypeIsValid(ListType::GetChildType(type)); + case LogicalTypeId::ARRAY: + return LogicalTypeIsValid(ArrayType::GetChildType(type)); + default: + return true; + } +} + int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) { - if (from.id() == LogicalTypeId::SQLNULL) { + if (from.id() == LogicalTypeId::SQLNULL || to.id() == LogicalTypeId::ANY) { // NULL expression can be cast to anything return TargetTypeCost(to); } @@ -200,9 +321,34 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) // parameter expression can be cast to anything for no cost return 0; } - if (to.id() == LogicalTypeId::ANY) { - // anything can be cast to ANY type for (almost no) cost - return 1; + if (from.id() == LogicalTypeId::STRING_LITERAL) { + // string literals can be cast to any type for low cost as long as the type is valid + // i.e. we cannot cast to LIST(ANY) as we don't know what "ANY" should be + // we cannot cast to DECIMAL without precision/width specified + if (!LogicalTypeIsValid(to)) { + return -1; + } + if (to.id() == LogicalTypeId::VARCHAR && to.GetAlias().empty()) { + return 1; + } + return 20; + } + if (from.id() == LogicalTypeId::INTEGER_LITERAL) { + // the integer literal has an underlying type - this type always matches + if (IntegerLiteral::GetType(from).id() == to.id()) { + return 0; + } + // integer literals can be cast to any other integer type for a low cost, but only if the literal fits + if (IntegerLiteral::FitsInType(from, to)) { + // to avoid ties we prefer BIGINT, INT, ... + auto target_cost = TargetTypeCost(to); + if (target_cost < 100) { + throw InternalException("Integer literal implicit cast - TargetTypeCost should be >= 100"); + } + return target_cost - 90; + } + // in any other case we use the casting rules of the preferred type of the literal + return CastRules::ImplicitCast(IntegerLiteral::GetType(from), to); } if (from.GetAlias() != to.GetAlias()) { // if aliases are different, an implicit cast is not possible @@ -211,7 +357,7 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) if (from.id() == LogicalTypeId::LIST && to.id() == LogicalTypeId::LIST) { // Lists can be cast if their child types can be cast auto child_cost = ImplicitCast(ListType::GetChildType(from), ListType::GetChildType(to)); - if (child_cost >= 100) { + if (child_cost >= 1) { // subtract one from the cost because we prefer LIST[X] -> LIST[VARCHAR] over LIST[X] -> VARCHAR child_cost--; } @@ -248,14 +394,6 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) // arguments match: do nothing return 0; } - if (from.id() == LogicalTypeId::BLOB && to.id() == LogicalTypeId::VARCHAR) { - // Implicit cast not allowed from BLOB to VARCHAR - return -1; - } - if (to.id() == LogicalTypeId::VARCHAR) { - // everything can be cast to VARCHAR, but this cast has a high cost - return TargetTypeCost(to); - } if (from.id() == LogicalTypeId::UNION && to.id() == LogicalTypeId::UNION) { // Unions can be cast if the source tags are a subset of the target tags @@ -301,18 +439,6 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) } } - if ((from.id() == LogicalTypeId::TIMESTAMP_SEC || from.id() == LogicalTypeId::TIMESTAMP_MS || - from.id() == LogicalTypeId::TIMESTAMP_NS) && - to.id() == LogicalTypeId::TIMESTAMP) { - //! Any timestamp type can be converted to the default (us) type at low cost - return 101; - } - if ((to.id() == LogicalTypeId::TIMESTAMP_SEC || to.id() == LogicalTypeId::TIMESTAMP_MS || - to.id() == LogicalTypeId::TIMESTAMP_NS) && - from.id() == LogicalTypeId::TIMESTAMP) { - //! Any timestamp type can be converted to the default (us) type at low cost - return 100; - } switch (from.id()) { case LogicalTypeId::TINYINT: return ImplicitCastTinyint(to); @@ -332,6 +458,8 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) return ImplicitCastUBigint(to); case LogicalTypeId::HUGEINT: return ImplicitCastHugeint(to); + case LogicalTypeId::UHUGEINT: + return ImplicitCastUhugeint(to); case LogicalTypeId::FLOAT: return ImplicitCastFloat(to); case LogicalTypeId::DOUBLE: @@ -340,6 +468,16 @@ int64_t CastRules::ImplicitCast(const LogicalType &from, const LogicalType &to) return ImplicitCastDate(to); case LogicalTypeId::DECIMAL: return ImplicitCastDecimal(to); + case LogicalTypeId::ENUM: + return ImplicitCastEnum(to); + case LogicalTypeId::TIMESTAMP_SEC: + return ImplicitCastTimestampSec(to); + case LogicalTypeId::TIMESTAMP_MS: + return ImplicitCastTimestampMS(to); + case LogicalTypeId::TIMESTAMP_NS: + return ImplicitCastTimestampNS(to); + case LogicalTypeId::TIMESTAMP: + return ImplicitCastTimestamp(to); default: return -1; } diff --git a/src/duckdb/src/function/compression_config.cpp b/src/duckdb/src/function/compression_config.cpp index 50e682970..177a2d86a 100644 --- a/src/duckdb/src/function/compression_config.cpp +++ b/src/duckdb/src/function/compression_config.cpp @@ -23,6 +23,8 @@ static DefaultCompressionMethod internal_compression_methods[] = { DictionaryCompressionFun::TypeIsSupported}, {CompressionType::COMPRESSION_CHIMP, ChimpCompressionFun::GetFunction, ChimpCompressionFun::TypeIsSupported}, {CompressionType::COMPRESSION_PATAS, PatasCompressionFun::GetFunction, PatasCompressionFun::TypeIsSupported}, + {CompressionType::COMPRESSION_ALP, AlpCompressionFun::GetFunction, AlpCompressionFun::TypeIsSupported}, + {CompressionType::COMPRESSION_ALPRD, AlpRDCompressionFun::GetFunction, AlpRDCompressionFun::TypeIsSupported}, {CompressionType::COMPRESSION_FSST, FSSTFun::GetFunction, FSSTFun::TypeIsSupported}, {CompressionType::COMPRESSION_AUTO, nullptr, nullptr}}; @@ -76,6 +78,8 @@ vector> DBConfig::GetCompressionFunctions(Physica TryLoadCompression(*this, result, CompressionType::COMPRESSION_DICTIONARY, data_type); TryLoadCompression(*this, result, CompressionType::COMPRESSION_CHIMP, data_type); TryLoadCompression(*this, result, CompressionType::COMPRESSION_PATAS, data_type); + TryLoadCompression(*this, result, CompressionType::COMPRESSION_ALP, data_type); + TryLoadCompression(*this, result, CompressionType::COMPRESSION_ALPRD, data_type); TryLoadCompression(*this, result, CompressionType::COMPRESSION_FSST, data_type); return result; } diff --git a/src/duckdb/src/function/function.cpp b/src/duckdb/src/function/function.cpp index 44547e888..3d76a7257 100644 --- a/src/duckdb/src/function/function.cpp +++ b/src/duckdb/src/function/function.cpp @@ -47,7 +47,7 @@ SimpleFunction::~SimpleFunction() { } string SimpleFunction::ToString() const { - return Function::CallToString(name, arguments); + return Function::CallToString(name, arguments, varargs); } bool SimpleFunction::HasVarArgs() const { @@ -71,17 +71,17 @@ bool SimpleNamedParameterFunction::HasNamedParameters() const { } BaseScalarFunction::BaseScalarFunction(string name_p, vector arguments_p, LogicalType return_type_p, - FunctionSideEffects side_effects, LogicalType varargs_p, + FunctionStability stability, LogicalType varargs_p, FunctionNullHandling null_handling) : SimpleFunction(std::move(name_p), std::move(arguments_p), std::move(varargs_p)), - return_type(std::move(return_type_p)), side_effects(side_effects), null_handling(null_handling) { + return_type(std::move(return_type_p)), stability(stability), null_handling(null_handling) { } BaseScalarFunction::~BaseScalarFunction() { } string BaseScalarFunction::ToString() const { - return Function::CallToString(name, arguments, return_type); + return Function::CallToString(name, arguments, varargs, return_type); } // add your initializer for new functions here @@ -118,16 +118,22 @@ hash_t BaseScalarFunction::Hash() const { return hash; } -string Function::CallToString(const string &name, const vector &arguments) { +string Function::CallToString(const string &name, const vector &arguments, const LogicalType &varargs) { string result = name + "("; - result += StringUtil::Join(arguments, arguments.size(), ", ", - [](const LogicalType &argument) { return argument.ToString(); }); + vector string_arguments; + for (auto &arg : arguments) { + string_arguments.push_back(arg.ToString()); + } + if (varargs.IsValid()) { + string_arguments.push_back("[" + varargs.ToString() + "...]"); + } + result += StringUtil::Join(string_arguments, ", "); return result + ")"; } -string Function::CallToString(const string &name, const vector &arguments, +string Function::CallToString(const string &name, const vector &arguments, const LogicalType &varargs, const LogicalType &return_type) { - string result = CallToString(name, arguments); + string result = CallToString(name, arguments, varargs); result += " -> " + return_type.ToString(); return result; } diff --git a/src/duckdb/src/function/function_binder.cpp b/src/duckdb/src/function/function_binder.cpp index 8b57bf92f..e6109d72e 100644 --- a/src/duckdb/src/function/function_binder.cpp +++ b/src/duckdb/src/function/function_binder.cpp @@ -6,6 +6,7 @@ #include "duckdb/execution/expression_executor.hpp" #include "duckdb/function/aggregate_function.hpp" #include "duckdb/function/cast_rules.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" #include "duckdb/planner/expression/bound_aggregate_expression.hpp" #include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression/bound_constant_expression.hpp" @@ -75,7 +76,7 @@ int64_t FunctionBinder::BindFunctionCost(const SimpleFunction &func, const vecto template vector FunctionBinder::BindFunctionsFromArguments(const string &name, FunctionSet &functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { idx_t best_function = DConstants::INVALID_INDEX; int64_t lowest_cost = NumericLimits::Maximum(); vector candidate_functions; @@ -100,14 +101,11 @@ vector FunctionBinder::BindFunctionsFromArguments(const string &name, Fun } if (best_function == DConstants::INVALID_INDEX) { // no matching function was found, throw an error - string call_str = Function::CallToString(name, arguments); - string candidate_str = ""; + vector candidates; for (auto &f : functions.functions) { - candidate_str += "\t" + f.ToString() + "\n"; + candidates.push_back(f.ToString()); } - error = StringUtil::Format("No function matches the given name and argument types '%s'. You might need to add " - "explicit type casts.\n\tCandidate functions:\n%s", - call_str, candidate_str); + error = ErrorData(BinderException::NoMatchingFunction(name, arguments, candidates)); return candidate_functions; } candidate_functions.push_back(best_function); @@ -117,25 +115,27 @@ vector FunctionBinder::BindFunctionsFromArguments(const string &name, Fun template idx_t FunctionBinder::MultipleCandidateException(const string &name, FunctionSet &functions, vector &candidate_functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { D_ASSERT(functions.functions.size() > 1); // there are multiple possible function definitions // throw an exception explaining which overloads are there string call_str = Function::CallToString(name, arguments); - string candidate_str = ""; + string candidate_str; for (auto &conf : candidate_functions) { T f = functions.GetFunctionByOffset(conf); candidate_str += "\t" + f.ToString() + "\n"; } - error = StringUtil::Format("Could not choose a best candidate function for the function call \"%s\". In order to " - "select one, please add explicit type casts.\n\tCandidate functions:\n%s", - call_str, candidate_str); + error = ErrorData( + ExceptionType::BINDER, + StringUtil::Format("Could not choose a best candidate function for the function call \"%s\". In order to " + "select one, please add explicit type casts.\n\tCandidate functions:\n%s", + call_str, candidate_str)); return DConstants::INVALID_INDEX; } template idx_t FunctionBinder::BindFunctionFromArguments(const string &name, FunctionSet &functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { auto candidate_functions = BindFunctionsFromArguments(name, functions, arguments, error); if (candidate_functions.empty()) { // no candidates @@ -158,29 +158,29 @@ idx_t FunctionBinder::BindFunctionFromArguments(const string &name, FunctionSet< } idx_t FunctionBinder::BindFunction(const string &name, ScalarFunctionSet &functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { return BindFunctionFromArguments(name, functions, arguments, error); } idx_t FunctionBinder::BindFunction(const string &name, AggregateFunctionSet &functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { return BindFunctionFromArguments(name, functions, arguments, error); } idx_t FunctionBinder::BindFunction(const string &name, TableFunctionSet &functions, - const vector &arguments, string &error) { + const vector &arguments, ErrorData &error) { return BindFunctionFromArguments(name, functions, arguments, error); } idx_t FunctionBinder::BindFunction(const string &name, PragmaFunctionSet &functions, vector ¶meters, - string &error) { + ErrorData &error) { vector types; for (auto &value : parameters) { types.push_back(value.type()); } idx_t entry = BindFunctionFromArguments(name, functions, types, error); if (entry == DConstants::INVALID_INDEX) { - throw BinderException(error); + error.Throw(); } auto candidate_function = functions.GetFunctionByOffset(entry); // cast the input parameters @@ -196,25 +196,25 @@ vector FunctionBinder::GetLogicalTypesFromExpressions(vector types; types.reserve(arguments.size()); for (auto &argument : arguments) { - types.push_back(argument->return_type); + types.push_back(ExpressionBinder::GetExpressionReturnType(*argument)); } return types; } idx_t FunctionBinder::BindFunction(const string &name, ScalarFunctionSet &functions, - vector> &arguments, string &error) { + vector> &arguments, ErrorData &error) { auto types = GetLogicalTypesFromExpressions(arguments); return BindFunction(name, functions, types, error); } idx_t FunctionBinder::BindFunction(const string &name, AggregateFunctionSet &functions, - vector> &arguments, string &error) { + vector> &arguments, ErrorData &error) { auto types = GetLogicalTypesFromExpressions(arguments); return BindFunction(name, functions, types, error); } idx_t FunctionBinder::BindFunction(const string &name, TableFunctionSet &functions, - vector> &arguments, string &error) { + vector> &arguments, ErrorData &error) { auto types = GetLogicalTypesFromExpressions(arguments); return BindFunction(name, functions, types, error); } @@ -237,9 +237,29 @@ LogicalTypeComparisonResult RequiresCast(const LogicalType &source_type, const L return LogicalTypeComparisonResult::DIFFERENT_TYPES; } +LogicalType PrepareTypeForCast(const LogicalType &type) { + if (type.id() == LogicalTypeId::ANY) { + return AnyType::GetTargetType(type); + } + if (type.id() == LogicalTypeId::LIST) { + return LogicalType::LIST(PrepareTypeForCast(ListType::GetChildType(type))); + } + return type; +} + void FunctionBinder::CastToFunctionArguments(SimpleFunction &function, vector> &children) { + for (auto &arg : function.arguments) { + arg = PrepareTypeForCast(arg); + } + function.varargs = PrepareTypeForCast(function.varargs); + for (idx_t i = 0; i < children.size(); i++) { auto target_type = i < function.arguments.size() ? function.arguments[i] : function.varargs; + if (target_type.id() == LogicalTypeId::STRING_LITERAL || target_type.id() == LogicalTypeId::INTEGER_LITERAL) { + throw InternalException( + "Function %s returned a STRING_LITERAL or INTEGER_LITERAL type - return an explicit type instead", + function.name); + } target_type.Verify(); // don't cast lambda children, they get removed before execution if (children[i]->return_type.id() == LogicalTypeId::LAMBDA) { @@ -257,7 +277,7 @@ void FunctionBinder::CastToFunctionArguments(SimpleFunction &function, vector FunctionBinder::BindScalarFunction(const string &schema, const string &name, - vector> children, string &error, + vector> children, ErrorData &error, bool is_operator, Binder *binder) { // bind the function auto &function = @@ -268,7 +288,7 @@ unique_ptr FunctionBinder::BindScalarFunction(const string &schema, } unique_ptr FunctionBinder::BindScalarFunction(ScalarFunctionCatalogEntry &func, - vector> children, string &error, + vector> children, ErrorData &error, bool is_operator, Binder *binder) { // bind the function idx_t best_function = BindFunction(func.name, func.functions, children, error); @@ -279,10 +299,28 @@ unique_ptr FunctionBinder::BindScalarFunction(ScalarFunctionCatalogE // found a matching function! auto bound_function = func.functions.GetFunctionByOffset(best_function); + // If any of the parameters are NULL, the function will just be replaced with a NULL constant + // But this NULL constant needs to have to correct type, because we use LogicalType::SQLNULL for binding macro's + // However, some functions may have an invalid return type, so we default to SQLNULL for those + LogicalType return_type_if_null; + switch (bound_function.return_type.id()) { + case LogicalTypeId::ANY: + case LogicalTypeId::DECIMAL: + case LogicalTypeId::STRUCT: + case LogicalTypeId::LIST: + case LogicalTypeId::MAP: + case LogicalTypeId::UNION: + case LogicalTypeId::ARRAY: + return_type_if_null = LogicalType::SQLNULL; + break; + default: + return_type_if_null = bound_function.return_type; + } + if (bound_function.null_handling == FunctionNullHandling::DEFAULT_NULL_HANDLING) { for (auto &child : children) { if (child->return_type == LogicalTypeId::SQLNULL) { - return make_uniq(Value(LogicalType::SQLNULL)); + return make_uniq(Value(return_type_if_null)); } if (!child->IsFoldable()) { continue; @@ -292,7 +330,7 @@ unique_ptr FunctionBinder::BindScalarFunction(ScalarFunctionCatalogE continue; } if (result.IsNull()) { - return make_uniq(Value(LogicalType::SQLNULL)); + return make_uniq(Value(return_type_if_null)); } } } diff --git a/src/duckdb/src/function/function_set.cpp b/src/duckdb/src/function/function_set.cpp index dbbfdfa8b..fd4cf3e70 100644 --- a/src/duckdb/src/function/function_set.cpp +++ b/src/duckdb/src/function/function_set.cpp @@ -14,12 +14,12 @@ ScalarFunctionSet::ScalarFunctionSet(ScalarFunction fun) : FunctionSet(std::move } ScalarFunction ScalarFunctionSet::GetFunctionByArguments(ClientContext &context, const vector &arguments) { - string error; + ErrorData error; FunctionBinder binder(context); idx_t index = binder.BindFunction(name, *this, arguments, error); if (index == DConstants::INVALID_INDEX) { throw InternalException("Failed to find function %s(%s)\n%s", name, StringUtil::ToString(arguments, ","), - error); + error.Message()); } return GetFunctionByOffset(index); } @@ -36,7 +36,7 @@ AggregateFunctionSet::AggregateFunctionSet(AggregateFunction fun) : FunctionSet( AggregateFunction AggregateFunctionSet::GetFunctionByArguments(ClientContext &context, const vector &arguments) { - string error; + ErrorData error; FunctionBinder binder(context); idx_t index = binder.BindFunction(name, *this, arguments, error); if (index == DConstants::INVALID_INDEX) { @@ -59,7 +59,7 @@ AggregateFunction AggregateFunctionSet::GetFunctionByArguments(ClientContext &co } } throw InternalException("Failed to find function %s(%s)\n%s", name, StringUtil::ToString(arguments, ","), - error); + error.Message()); } return GetFunctionByOffset(index); } @@ -72,12 +72,12 @@ TableFunctionSet::TableFunctionSet(TableFunction fun) : FunctionSet(std::move(fu } TableFunction TableFunctionSet::GetFunctionByArguments(ClientContext &context, const vector &arguments) { - string error; + ErrorData error; FunctionBinder binder(context); idx_t index = binder.BindFunction(name, *this, arguments, error); if (index == DConstants::INVALID_INDEX) { throw InternalException("Failed to find function %s(%s)\n%s", name, StringUtil::ToString(arguments, ","), - error); + error.Message()); } return GetFunctionByOffset(index); } diff --git a/src/duckdb/src/function/pragma/pragma_functions.cpp b/src/duckdb/src/function/pragma/pragma_functions.cpp index e312ab02b..d44ec629d 100644 --- a/src/duckdb/src/function/pragma/pragma_functions.cpp +++ b/src/duckdb/src/function/pragma/pragma_functions.cpp @@ -5,6 +5,7 @@ #include "duckdb/main/client_context.hpp" #include "duckdb/main/database.hpp" #include "duckdb/main/query_profiler.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/planner/expression_binder.hpp" #include "duckdb/storage/buffer_manager.hpp" @@ -76,6 +77,14 @@ static void PragmaDisableExternalVerification(ClientContext &context, const Func ClientConfig::GetConfig(context).verify_external = false; } +static void PragmaEnableFetchRowVerification(ClientContext &context, const FunctionParameters ¶meters) { + ClientConfig::GetConfig(context).verify_fetch_row = true; +} + +static void PragmaDisableFetchRowVerification(ClientContext &context, const FunctionParameters ¶meters) { + ClientConfig::GetConfig(context).verify_fetch_row = false; +} + static void PragmaEnableForceParallelism(ClientContext &context, const FunctionParameters ¶meters) { ClientConfig::GetConfig(context).verify_parallelism = true; } @@ -124,6 +133,9 @@ void PragmaFunctions::RegisterFunction(BuiltinFunctions &set) { set.AddFunction(PragmaFunction::PragmaStatement("verify_external", PragmaEnableExternalVerification)); set.AddFunction(PragmaFunction::PragmaStatement("disable_verify_external", PragmaDisableExternalVerification)); + set.AddFunction(PragmaFunction::PragmaStatement("verify_fetch_row", PragmaEnableFetchRowVerification)); + set.AddFunction(PragmaFunction::PragmaStatement("disable_verify_fetch_row", PragmaDisableFetchRowVerification)); + set.AddFunction(PragmaFunction::PragmaStatement("verify_serializer", PragmaVerifySerializer)); set.AddFunction(PragmaFunction::PragmaStatement("disable_verify_serializer", PragmaDisableVerifySerializer)); diff --git a/src/duckdb/src/function/pragma/pragma_queries.cpp b/src/duckdb/src/function/pragma/pragma_queries.cpp index 10b0b8b77..6b1b180c0 100644 --- a/src/duckdb/src/function/pragma/pragma_queries.cpp +++ b/src/duckdb/src/function/pragma/pragma_queries.cpp @@ -14,10 +14,11 @@ namespace duckdb { string PragmaTableInfo(ClientContext &context, const FunctionParameters ¶meters) { - return StringUtil::Format("SELECT * FROM pragma_table_info('%s');", parameters.values[0].ToString()); + return StringUtil::Format("SELECT * FROM pragma_table_info(%s);", + KeywordHelper::WriteQuoted(parameters.values[0].ToString(), '\'')); } -string PragmaShowTables(ClientContext &context, const FunctionParameters ¶meters) { +string PragmaShowTables() { // clang-format off return R"EOF( with "tables" as @@ -41,8 +42,11 @@ string PragmaShowTables(ClientContext &context, const FunctionParameters ¶me ORDER BY "name";)EOF"; // clang-format on } +string PragmaShowTables(ClientContext &context, const FunctionParameters ¶meters) { + return PragmaShowTables(); +} -string PragmaShowTablesExpanded(ClientContext &context, const FunctionParameters ¶meters) { +string PragmaShowTablesExpanded() { return R"( SELECT t.database_name AS database, @@ -74,10 +78,18 @@ string PragmaShowTablesExpanded(ClientContext &context, const FunctionParameters )"; } -string PragmaShowDatabases(ClientContext &context, const FunctionParameters ¶meters) { +string PragmaShowTablesExpanded(ClientContext &context, const FunctionParameters ¶meters) { + return PragmaShowTablesExpanded(); +} + +string PragmaShowDatabases() { return "SELECT database_name FROM duckdb_databases() WHERE NOT internal ORDER BY database_name;"; } +string PragmaShowDatabases(ClientContext &context, const FunctionParameters ¶meters) { + return PragmaShowDatabases(); +} + string PragmaAllProfiling(ClientContext &context, const FunctionParameters ¶meters) { return "SELECT * FROM pragma_last_profiling_output() JOIN pragma_detailed_profiling_output() ON " "(pragma_last_profiling_output.operator_id);"; @@ -99,41 +111,12 @@ string PragmaFunctionsQuery(ClientContext &context, const FunctionParameters &pa " ORDER BY 1;"; } -string PragmaShow(ClientContext &context, const FunctionParameters ¶meters) { - // PRAGMA table_info but with some aliases - auto table = QualifiedName::Parse(parameters.values[0].ToString()); - - // clang-format off - string sql = R"( - SELECT - name AS "column_name", - type as "column_type", - CASE WHEN "notnull" THEN 'NO' ELSE 'YES' END AS "null", - (SELECT - MIN(CASE - WHEN constraint_type='PRIMARY KEY' THEN 'PRI' - WHEN constraint_type='UNIQUE' THEN 'UNI' - ELSE NULL END) - FROM duckdb_constraints() c - WHERE c.table_oid=cols.table_oid - AND list_contains(constraint_column_names, cols.column_name)) AS "key", - dflt_value AS "default", - NULL AS "extra" - FROM pragma_table_info('%func_param_table%') - LEFT JOIN duckdb_columns cols - ON cols.column_name = pragma_table_info.name - AND cols.table_name='%table_name%' - AND cols.schema_name='%table_schema%' - AND cols.database_name = '%table_database%' - ORDER BY column_index;)"; - // clang-format on +string PragmaShow(const string &table_name) { + return StringUtil::Format("SELECT * FROM pragma_show(%s);", KeywordHelper::WriteQuoted(table_name, '\'')); +} - sql = StringUtil::Replace(sql, "%func_param_table%", parameters.values[0].ToString()); - sql = StringUtil::Replace(sql, "%table_name%", table.name); - sql = StringUtil::Replace(sql, "%table_schema%", table.schema.empty() ? DEFAULT_SCHEMA : table.schema); - sql = StringUtil::Replace(sql, "%table_database%", - table.catalog.empty() ? DatabaseManager::GetDefaultDatabase(context) : table.catalog); - return sql; +string PragmaShow(ClientContext &context, const FunctionParameters ¶meters) { + return PragmaShow(parameters.values[0].ToString()); } string PragmaVersion(ClientContext &context, const FunctionParameters ¶meters) { diff --git a/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp b/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp index 173353fb8..9da8d10d5 100644 --- a/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp +++ b/src/duckdb/src/function/scalar/compressed_materialization/compress_integral.cpp @@ -26,6 +26,14 @@ struct TemplatedIntegralCompress { } }; +template +struct TemplatedIntegralCompress { + static inline RESULT_TYPE Operation(const uhugeint_t &input, const uhugeint_t &min_val) { + D_ASSERT(min_val <= input); + return (input - min_val).lower; + } +}; + template static void IntegralCompressFunction(DataChunk &args, ExpressionState &state, Vector &result) { D_ASSERT(args.ColumnCount() == 2); @@ -75,6 +83,8 @@ static scalar_function_t GetIntegralCompressFunctionInputSwitch(const LogicalTyp return GetIntegralCompressFunctionResultSwitch(input_type, result_type); case LogicalTypeId::UBIGINT: return GetIntegralCompressFunctionResultSwitch(input_type, result_type); + case LogicalTypeId::UHUGEINT: + return GetIntegralCompressFunctionResultSwitch(input_type, result_type); default: throw InternalException("Unexpected input type in GetIntegralCompressFunctionInputSwitch"); } @@ -124,6 +134,8 @@ static scalar_function_t GetIntegralDecompressFunctionResultSwitch(const Logical return GetIntegralDecompressFunction(input_type, result_type); case LogicalTypeId::UBIGINT: return GetIntegralDecompressFunction(input_type, result_type); + case LogicalTypeId::UHUGEINT: + return GetIntegralDecompressFunction(input_type, result_type); default: throw InternalException("Unexpected input type in GetIntegralDecompressFunctionSetSwitch"); } 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 3b9614359..b7f454581 100644 --- a/src/duckdb/src/function/scalar/list/contains_or_position.cpp +++ b/src/duckdb/src/function/scalar/list/contains_or_position.cpp @@ -41,7 +41,12 @@ static unique_ptr ListContainsOrPositionBind(ClientContext &contex bound_function.return_type = RETURN_TYPE; } else { auto const &child_type = ListType::GetChildType(list); - auto max_child_type = LogicalType::MaxLogicalType(child_type, value); + LogicalType max_child_type; + if (!LogicalType::TryGetMaxLogicalType(context, child_type, value, max_child_type)) { + throw BinderException( + "Cannot get list_position of element of type %s in a list of type %s[] - an explicit cast is required", + value.ToString(), child_type.ToString()); + } auto list_type = LogicalType::LIST(max_child_type); bound_function.arguments[0] = list_type; diff --git a/src/duckdb/src/function/scalar/list/list_concat.cpp b/src/duckdb/src/function/scalar/list/list_concat.cpp index d992bd951..880d75264 100644 --- a/src/duckdb/src/function/scalar/list/list_concat.cpp +++ b/src/duckdb/src/function/scalar/list/list_concat.cpp @@ -99,7 +99,11 @@ static unique_ptr ListConcatBind(ClientContext &context, ScalarFun // Resolve list type LogicalType child_type = LogicalType::SQLNULL; for (const auto &argument : arguments) { - child_type = LogicalType::MaxLogicalType(child_type, ListType::GetChildType(argument->return_type)); + auto &next_type = ListType::GetChildType(argument->return_type); + if (!LogicalType::TryGetMaxLogicalType(context, child_type, next_type, child_type)) { + throw BinderException("Cannot concatenate lists of types %s[] and %s[] - an explicit cast is required", + child_type.ToString(), next_type.ToString()); + } } auto list_type = LogicalType::LIST(child_type); diff --git a/src/duckdb/src/function/scalar/list/list_extract.cpp b/src/duckdb/src/function/scalar/list/list_extract.cpp index cb0ffb5ad..99784f975 100644 --- a/src/duckdb/src/function/scalar/list/list_extract.cpp +++ b/src/duckdb/src/function/scalar/list/list_extract.cpp @@ -1,7 +1,8 @@ #include "duckdb/common/pair.hpp" #include "duckdb/common/string_util.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/common/types/data_chunk.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/vector_operations/binary_executor.hpp" #include "duckdb/function/scalar/nested_functions.hpp" #include "duckdb/function/scalar/string_functions.hpp" @@ -114,6 +115,9 @@ static void ExecuteListExtractInternal(const idx_t count, UnifiedVectorFormat &l case PhysicalType::UINT64: ListExtractTemplate(count, list, offsets, child_vector, list_size, result); break; + case PhysicalType::UINT128: + ListExtractTemplate(count, list, offsets, child_vector, list_size, result); + break; case PhysicalType::FLOAT: ListExtractTemplate(count, list, offsets, child_vector, list_size, result); break; diff --git a/src/duckdb/src/function/scalar/operators/add.cpp b/src/duckdb/src/function/scalar/operators/add.cpp index 6dcde0637..f0670120f 100644 --- a/src/duckdb/src/function/scalar/operators/add.cpp +++ b/src/duckdb/src/function/scalar/operators/add.cpp @@ -7,6 +7,7 @@ #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" namespace duckdb { @@ -68,19 +69,43 @@ timestamp_t AddOperator::Operation(date_t left, dtime_t right) { return result; } +template <> +timestamp_t AddOperator::Operation(date_t left, dtime_tz_t right) { + if (left == date_t::infinity()) { + return timestamp_t::infinity(); + } else if (left == date_t::ninfinity()) { + return timestamp_t::ninfinity(); + } + timestamp_t result; + if (!Timestamp::TryFromDatetime(left, right, result)) { + throw OutOfRangeException("Timestamp with time zone out of range"); + } + return result; +} + template <> timestamp_t AddOperator::Operation(dtime_t left, date_t right) { return AddOperator::Operation(right, left); } template <> -date_t AddOperator::Operation(date_t left, interval_t right) { - return Interval::Add(left, right); +timestamp_t AddOperator::Operation(dtime_tz_t left, date_t right) { + return AddOperator::Operation(right, left); } template <> -date_t AddOperator::Operation(interval_t left, date_t right) { - return AddOperator::Operation(right, left); +timestamp_t AddOperator::Operation(date_t left, interval_t right) { + if (left == date_t::infinity()) { + return timestamp_t::infinity(); + } else if (left == date_t::ninfinity()) { + return timestamp_t::ninfinity(); + } + return Interval::Add(Timestamp::FromDatetime(left, dtime_t(0)), right); +} + +template <> +timestamp_t AddOperator::Operation(interval_t left, date_t right) { + return AddOperator::Operation(right, left); } template <> @@ -160,9 +185,18 @@ bool TryAddOperator::Operation(int64_t left, int64_t right, int64_t &result) { return true; } +template <> +bool TryAddOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result) { + if (!Uhugeint::TryAddInPlace(left, right)) { + return false; + } + result = left; + return true; +} + template <> bool TryAddOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t &result) { - if (!Hugeint::AddInPlace(left, right)) { + if (!Hugeint::TryAddInPlace(left, right)) { return false; } result = left; @@ -204,7 +238,9 @@ bool TryDecimalAdd::Operation(int64_t left, int64_t right, int64_t &result) { template <> bool TryDecimalAdd::Operation(hugeint_t left, hugeint_t right, hugeint_t &result) { - result = left + right; + if (!TryAddOperator::Operation(left, right, result)) { + return false; + } if (result <= -Hugeint::POWERS_OF_TEN[38] || result >= Hugeint::POWERS_OF_TEN[38]) { return false; } @@ -234,4 +270,15 @@ dtime_t AddTimeOperator::Operation(interval_t left, dtime_t right) { return AddTimeOperator::Operation(right, left); } +template <> +dtime_tz_t AddTimeOperator::Operation(dtime_tz_t left, interval_t right) { + date_t date(0); + return Interval::Add(left, right, date); +} + +template <> +dtime_tz_t AddTimeOperator::Operation(interval_t left, dtime_tz_t right) { + return AddTimeOperator::Operation(right, left); +} + } // namespace duckdb diff --git a/src/duckdb/src/function/scalar/operators/arithmetic.cpp b/src/duckdb/src/function/scalar/operators/arithmetic.cpp index f316fb0a4..ebb76e30e 100644 --- a/src/duckdb/src/function/scalar/operators/arithmetic.cpp +++ b/src/duckdb/src/function/scalar/operators/arithmetic.cpp @@ -49,8 +49,11 @@ static scalar_function_t GetScalarIntegerFunction(PhysicalType type) { case PhysicalType::UINT64: function = &ScalarFunction::BinaryFunction; break; + case PhysicalType::UINT128: + function = &ScalarFunction::BinaryFunction; + break; default: - throw NotImplementedException("Unimplemented type for GetScalarBinaryFunction"); + throw NotImplementedException("Unimplemented type for GetScalarBinaryFunction: %s", TypeIdToString(type)); } return function; } @@ -59,9 +62,6 @@ template static scalar_function_t GetScalarBinaryFunction(PhysicalType type) { scalar_function_t function; switch (type) { - case PhysicalType::INT128: - function = &ScalarFunction::BinaryFunction; - break; case PhysicalType::FLOAT: function = &ScalarFunction::BinaryFunction; break; @@ -238,7 +238,7 @@ unique_ptr BindDecimalAddSubtract(ClientContext &context, ScalarFu } else { bound_function.function = GetScalarBinaryFunction(result_type.InternalType()); } - if (result_type.InternalType() != PhysicalType::INT128) { + if (result_type.InternalType() != PhysicalType::INT128 && result_type.InternalType() != PhysicalType::UINT128) { if (IS_SUBTRACT) { bound_function.statistics = PropagateNumericStats; @@ -319,11 +319,14 @@ ScalarFunction AddFun::GetFunction(const LogicalType &left_type, const LogicalTy return ScalarFunction("+", {left_type, right_type}, LogicalType::DATE, ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::INTERVAL) { - return ScalarFunction("+", {left_type, right_type}, LogicalType::DATE, - ScalarFunction::BinaryFunction); + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP, + ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::TIME) { return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP, ScalarFunction::BinaryFunction); + } else if (right_type.id() == LogicalTypeId::TIME_TZ) { + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP_TZ, + ScalarFunction::BinaryFunction); } break; case LogicalTypeId::INTEGER: @@ -337,11 +340,14 @@ ScalarFunction AddFun::GetFunction(const LogicalType &left_type, const LogicalTy return ScalarFunction("+", {left_type, right_type}, LogicalType::INTERVAL, ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::DATE) { - return ScalarFunction("+", {left_type, right_type}, LogicalType::DATE, - ScalarFunction::BinaryFunction); + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP, + ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::TIME) { return ScalarFunction("+", {left_type, right_type}, LogicalType::TIME, ScalarFunction::BinaryFunction); + } else if (right_type.id() == LogicalTypeId::TIME_TZ) { + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIME_TZ, + ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::TIMESTAMP) { return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP, ScalarFunction::BinaryFunction); @@ -356,6 +362,15 @@ ScalarFunction AddFun::GetFunction(const LogicalType &left_type, const LogicalTy ScalarFunction::BinaryFunction); } break; + case LogicalTypeId::TIME_TZ: + if (right_type.id() == LogicalTypeId::DATE) { + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP_TZ, + ScalarFunction::BinaryFunction); + } else if (right_type.id() == LogicalTypeId::INTERVAL) { + return ScalarFunction("+", {left_type, right_type}, LogicalType::TIME_TZ, + ScalarFunction::BinaryFunction); + } + break; case LogicalTypeId::TIMESTAMP: if (right_type.id() == LogicalTypeId::INTERVAL) { return ScalarFunction("+", {left_type, right_type}, LogicalType::TIMESTAMP, @@ -394,10 +409,17 @@ void AddFun::RegisterFunction(BuiltinFunctions &set) { functions.AddFunction(GetFunction(LogicalType::TIMESTAMP, LogicalType::INTERVAL)); functions.AddFunction(GetFunction(LogicalType::INTERVAL, LogicalType::TIMESTAMP)); + functions.AddFunction(GetFunction(LogicalType::TIME_TZ, LogicalType::INTERVAL)); + functions.AddFunction(GetFunction(LogicalType::INTERVAL, LogicalType::TIME_TZ)); + // we can add times to dates functions.AddFunction(GetFunction(LogicalType::TIME, LogicalType::DATE)); functions.AddFunction(GetFunction(LogicalType::DATE, LogicalType::TIME)); + // we can add times with time zones (offsets) to dates + functions.AddFunction(GetFunction(LogicalType::TIME_TZ, LogicalType::DATE)); + functions.AddFunction(GetFunction(LogicalType::DATE, LogicalType::TIME_TZ)); + // we can add lists together functions.AddFunction(ListConcatFun::GetFunction()); @@ -587,8 +609,8 @@ ScalarFunction SubtractFun::GetFunction(const LogicalType &left_type, const Logi return ScalarFunction("-", {left_type, right_type}, LogicalType::DATE, ScalarFunction::BinaryFunction); } else if (right_type.id() == LogicalTypeId::INTERVAL) { - return ScalarFunction("-", {left_type, right_type}, LogicalType::DATE, - ScalarFunction::BinaryFunction); + return ScalarFunction("-", {left_type, right_type}, LogicalType::TIMESTAMP, + ScalarFunction::BinaryFunction); } break; case LogicalTypeId::TIMESTAMP: @@ -614,6 +636,13 @@ ScalarFunction SubtractFun::GetFunction(const LogicalType &left_type, const Logi ScalarFunction::BinaryFunction); } break; + case LogicalTypeId::TIME_TZ: + if (right_type.id() == LogicalTypeId::INTERVAL) { + return ScalarFunction( + "-", {left_type, right_type}, LogicalType::TIME_TZ, + ScalarFunction::BinaryFunction); + } + break; default: break; } @@ -643,6 +672,7 @@ void SubtractFun::RegisterFunction(BuiltinFunctions &set) { functions.AddFunction(GetFunction(LogicalType::DATE, LogicalType::INTERVAL)); functions.AddFunction(GetFunction(LogicalType::TIME, LogicalType::INTERVAL)); functions.AddFunction(GetFunction(LogicalType::TIMESTAMP, LogicalType::INTERVAL)); + functions.AddFunction(GetFunction(LogicalType::TIME_TZ, LogicalType::INTERVAL)); // we can negate intervals functions.AddFunction(GetFunction(LogicalType::INTERVAL)); set.AddFunction(functions); @@ -856,10 +886,12 @@ struct BinaryZeroIsNullWrapper { } }; -struct BinaryZeroIsNullHugeintWrapper { +struct BinaryNumericDivideHugeintWrapper { template static inline RESULT_TYPE Operation(FUNC fun, LEFT_TYPE left, RIGHT_TYPE right, ValidityMask &mask, idx_t idx) { - if (right.upper == 0 && right.lower == 0) { + if (left == NumericLimits::Minimum() && right == -1) { + throw OutOfRangeException("Overflow in division of %s / %s", left.ToString(), right.ToString()); + } else if (right == 0) { mask.SetInvalid(idx); return left; } else { @@ -897,7 +929,9 @@ static scalar_function_t GetBinaryFunctionIgnoreZero(const LogicalType &type) { case LogicalTypeId::UBIGINT: return BinaryScalarFunctionIgnoreZero; case LogicalTypeId::HUGEINT: - return BinaryScalarFunctionIgnoreZero; + return BinaryScalarFunctionIgnoreZero; + case LogicalTypeId::UHUGEINT: + return BinaryScalarFunctionIgnoreZero; case LogicalTypeId::FLOAT: return BinaryScalarFunctionIgnoreZero; case LogicalTypeId::DOUBLE: diff --git a/src/duckdb/src/function/scalar/operators/multiply.cpp b/src/duckdb/src/function/scalar/operators/multiply.cpp index 718f9a8e4..b0e033a97 100644 --- a/src/duckdb/src/function/scalar/operators/multiply.cpp +++ b/src/duckdb/src/function/scalar/operators/multiply.cpp @@ -2,6 +2,7 @@ #include "duckdb/common/limits.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/value.hpp" #include "duckdb/common/windows_undefs.hpp" @@ -183,6 +184,11 @@ bool TryMultiplyOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t & return Hugeint::TryMultiply(left, right, result); } +template <> +bool TryMultiplyOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result) { + return Uhugeint::TryMultiply(left, right, result); +} + //===--------------------------------------------------------------------===// // multiply decimal with overflow check //===--------------------------------------------------------------------===// @@ -211,7 +217,9 @@ bool TryDecimalMultiply::Operation(int64_t left, int64_t right, int64_t &result) template <> bool TryDecimalMultiply::Operation(hugeint_t left, hugeint_t right, hugeint_t &result) { - result = left * right; + if (!TryMultiplyOperator::Operation(left, right, result)) { + return false; + } if (result <= -Hugeint::POWERS_OF_TEN[38] || result >= Hugeint::POWERS_OF_TEN[38]) { return false; } diff --git a/src/duckdb/src/function/scalar/operators/subtract.cpp b/src/duckdb/src/function/scalar/operators/subtract.cpp index c9918a386..4d4982312 100644 --- a/src/duckdb/src/function/scalar/operators/subtract.cpp +++ b/src/duckdb/src/function/scalar/operators/subtract.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/limits.hpp" #include "duckdb/common/operator/add.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/date.hpp" #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/value.hpp" @@ -56,8 +57,8 @@ interval_t SubtractOperator::Operation(interval_t left, interval_t right) { } template <> -date_t SubtractOperator::Operation(date_t left, interval_t right) { - return AddOperator::Operation(left, Interval::Invert(right)); +timestamp_t SubtractOperator::Operation(date_t left, interval_t right) { + return AddOperator::Operation(left, Interval::Invert(right)); } template <> @@ -156,7 +157,13 @@ bool TrySubtractOperator::Operation(int64_t left, int64_t right, int64_t &result template <> bool TrySubtractOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t &result) { result = left; - return Hugeint::SubtractInPlace(result, right); + return Hugeint::TrySubtractInPlace(result, right); +} + +template <> +bool TrySubtractOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result) { + result = left; + return Uhugeint::TrySubtractInPlace(result, right); } //===--------------------------------------------------------------------===// @@ -194,7 +201,9 @@ bool TryDecimalSubtract::Operation(int64_t left, int64_t right, int64_t &result) template <> bool TryDecimalSubtract::Operation(hugeint_t left, hugeint_t right, hugeint_t &result) { - result = left - right; + if (!TrySubtractOperator::Operation(left, right, result)) { + return false; + } if (result <= -Hugeint::POWERS_OF_TEN[38] || result >= Hugeint::POWERS_OF_TEN[38]) { return false; } @@ -219,4 +228,10 @@ dtime_t SubtractTimeOperator::Operation(dtime_t left, interval_t right) { return AddTimeOperator::Operation(left, right); } +template <> +dtime_tz_t SubtractTimeOperator::Operation(dtime_tz_t left, interval_t right) { + right.micros = -right.micros; + return AddTimeOperator::Operation(left, right); +} + } // namespace duckdb diff --git a/src/duckdb/src/function/scalar/sequence/nextval.cpp b/src/duckdb/src/function/scalar/sequence/nextval.cpp index 9b54e7ab0..8fba14654 100644 --- a/src/duckdb/src/function/scalar/sequence/nextval.cpp +++ b/src/duckdb/src/function/scalar/sequence/nextval.cpp @@ -8,71 +8,22 @@ #include "duckdb/execution/expression_executor.hpp" #include "duckdb/planner/expression/bound_function_expression.hpp" #include "duckdb/transaction/duck_transaction.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/vector_operations/unary_executor.hpp" -#include "duckdb/common/operator/add.hpp" #include "duckdb/planner/binder.hpp" namespace duckdb { -struct NextvalBindData : public FunctionData { - explicit NextvalBindData(optional_ptr sequence) : sequence(sequence) { - } - - //! The sequence to use for the nextval computation; only if the sequence is a constant - optional_ptr sequence; - - unique_ptr Copy() const override { - return make_uniq(sequence); - } - - bool Equals(const FunctionData &other_p) const override { - auto &other = other_p.Cast(); - return sequence == other.sequence; - } -}; - struct CurrentSequenceValueOperator { static int64_t Operation(DuckTransaction &transaction, SequenceCatalogEntry &seq) { - lock_guard seqlock(seq.lock); - int64_t result; - if (seq.usage_count == 0u) { - throw SequenceException("currval: sequence is not yet defined in this session"); - } - result = seq.last_value; - return result; + return seq.CurrentValue(); } }; struct NextSequenceValueOperator { static int64_t Operation(DuckTransaction &transaction, SequenceCatalogEntry &seq) { - lock_guard seqlock(seq.lock); - int64_t result; - result = seq.counter; - bool overflow = !TryAddOperator::Operation(seq.counter, seq.increment, seq.counter); - if (seq.cycle) { - if (overflow) { - seq.counter = seq.increment < 0 ? seq.max_value : seq.min_value; - } else if (seq.counter < seq.min_value) { - seq.counter = seq.max_value; - } else if (seq.counter > seq.max_value) { - seq.counter = seq.min_value; - } - } else { - if (result < seq.min_value || (overflow && seq.increment < 0)) { - throw SequenceException("nextval: reached minimum value of sequence \"%s\" (%lld)", seq.name, - seq.min_value); - } - if (result > seq.max_value || overflow) { - throw SequenceException("nextval: reached maximum value of sequence \"%s\" (%lld)", seq.name, - seq.max_value); - } - } - seq.last_value = result; - seq.usage_count++; - if (!seq.temporary) { - transaction.sequence_usage[&seq] = SequenceValue(seq.usage_count, seq.counter); - } - return result; + return seq.NextValue(transaction); } }; @@ -134,17 +85,39 @@ static void NextValDependency(BoundFunctionExpression &expr, DependencyList &dep } } +void Serialize(Serializer &serializer, const optional_ptr bind_data, const ScalarFunction &) { + auto &next_val_bind_data = bind_data->Cast(); + serializer.WritePropertyWithDefault(100, "sequence_create_info", next_val_bind_data.create_info); +} + +unique_ptr Deserialize(Deserializer &deserializer, ScalarFunction &) { + auto create_info = deserializer.ReadPropertyWithDefault>(100, "sequence_create_info", + unique_ptr()); + optional_ptr catalog_entry_ptr; + if (create_info) { + auto &seq_info = create_info->Cast(); + auto &context = deserializer.Get(); + catalog_entry_ptr = + &Catalog::GetEntry(context, seq_info.catalog, seq_info.schema, seq_info.name); + } + return make_uniq(catalog_entry_ptr); +} + void NextvalFun::RegisterFunction(BuiltinFunctions &set) { ScalarFunction next_val("nextval", {LogicalType::VARCHAR}, LogicalType::BIGINT, NextValFunction, NextValBind, NextValDependency); - next_val.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + next_val.stability = FunctionStability::VOLATILE; + next_val.serialize = Serialize; + next_val.deserialize = Deserialize; set.AddFunction(next_val); } void CurrvalFun::RegisterFunction(BuiltinFunctions &set) { ScalarFunction curr_val("currval", {LogicalType::VARCHAR}, LogicalType::BIGINT, NextValFunction, NextValBind, NextValDependency); - curr_val.side_effects = FunctionSideEffects::HAS_SIDE_EFFECTS; + curr_val.stability = FunctionStability::VOLATILE; + curr_val.serialize = Serialize; + curr_val.deserialize = Deserialize; set.AddFunction(curr_val); } diff --git a/src/duckdb/src/function/scalar/string/concat.cpp b/src/duckdb/src/function/scalar/string/concat.cpp index f5b04a4b8..5ad0c9a3e 100644 --- a/src/duckdb/src/function/scalar/string/concat.cpp +++ b/src/duckdb/src/function/scalar/string/concat.cpp @@ -231,6 +231,15 @@ static void ConcatWSFunction(DataChunk &args, ExpressionState &state, Vector &re } } +static unique_ptr BindConcatFunction(ClientContext &context, ScalarFunction &bound_function, + vector> &arguments) { + for (auto &arg : bound_function.arguments) { + arg = LogicalType::VARCHAR; + } + bound_function.varargs = LogicalType::VARCHAR; + return nullptr; +} + void ConcatFun::RegisterFunction(BuiltinFunctions &set) { // the concat operator and concat function have different behavior regarding NULLs // this is strange but seems consistent with postgresql and mysql @@ -247,14 +256,15 @@ void ConcatFun::RegisterFunction(BuiltinFunctions &set) { // e.g.: // concat_ws(',', NULL, NULL) = "" // concat_ws(',', '', '') = "," - ScalarFunction concat = ScalarFunction("concat", {LogicalType::VARCHAR}, LogicalType::VARCHAR, ConcatFunction); - concat.varargs = LogicalType::VARCHAR; + ScalarFunction concat = + ScalarFunction("concat", {LogicalType::ANY}, LogicalType::VARCHAR, ConcatFunction, BindConcatFunction); + concat.varargs = LogicalType::ANY; concat.null_handling = FunctionNullHandling::SPECIAL_HANDLING; set.AddFunction(concat); ScalarFunctionSet concat_op("||"); concat_op.AddFunction( - ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::VARCHAR, ConcatOperator)); + ScalarFunction({LogicalType::ANY, LogicalType::ANY}, LogicalType::VARCHAR, ConcatOperator, BindConcatFunction)); concat_op.AddFunction(ScalarFunction({LogicalType::BLOB, LogicalType::BLOB}, LogicalType::BLOB, ConcatOperator)); concat_op.AddFunction(ListConcatFun::GetFunction()); for (auto &fun : concat_op.functions) { @@ -262,9 +272,9 @@ void ConcatFun::RegisterFunction(BuiltinFunctions &set) { } set.AddFunction(concat_op); - ScalarFunction concat_ws = ScalarFunction("concat_ws", {LogicalType::VARCHAR, LogicalType::VARCHAR}, - LogicalType::VARCHAR, ConcatWSFunction); - concat_ws.varargs = LogicalType::VARCHAR; + ScalarFunction concat_ws = ScalarFunction("concat_ws", {LogicalType::VARCHAR, LogicalType::ANY}, + LogicalType::VARCHAR, ConcatWSFunction, BindConcatFunction); + concat_ws.varargs = LogicalType::ANY; concat_ws.null_handling = FunctionNullHandling::SPECIAL_HANDLING; set.AddFunction(concat_ws); } diff --git a/src/duckdb/src/function/scalar/string/regexp.cpp b/src/duckdb/src/function/scalar/string/regexp.cpp index d14ac7ac5..89aa0e30d 100644 --- a/src/duckdb/src/function/scalar/string/regexp.cpp +++ b/src/duckdb/src/function/scalar/string/regexp.cpp @@ -55,7 +55,7 @@ RegexpMatchesBindData::RegexpMatchesBindData(duckdb_re2::RE2::Options options, s if (constant_pattern) { auto pattern = make_uniq(constant_string, options); if (!pattern->ok()) { - throw Exception(pattern->error()); + throw InvalidInputException(pattern->error()); } range_success = pattern->PossibleMatchRange(&range_min, &range_max, 1000); @@ -122,7 +122,7 @@ static void RegexpMatchesFunction(DataChunk &args, ExpressionState &state, Vecto [&](string_t input, string_t pattern) { RE2 re(CreateStringPiece(pattern), info.options); if (!re.ok()) { - throw Exception(re.error()); + throw InvalidInputException(re.error()); } return OP::Operation(CreateStringPiece(input), re); }); @@ -382,24 +382,24 @@ static unique_ptr RegexExtractBind(ClientContext &context, ScalarF void RegexpFun::RegisterFunction(BuiltinFunctions &set) { ScalarFunctionSet regexp_full_match("regexp_full_match"); - regexp_full_match.AddFunction(ScalarFunction( - {LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, RegexpMatchesFunction, - RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); - regexp_full_match.AddFunction(ScalarFunction( - {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, - RegexpMatchesFunction, RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, - LogicalType::INVALID, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + regexp_full_match.AddFunction( + ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, + RegexpMatchesFunction, RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); + regexp_full_match.AddFunction( + ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, + RegexpMatchesFunction, RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); ScalarFunctionSet regexp_partial_match("regexp_matches"); regexp_partial_match.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, RegexpMatchesFunction, - RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING)); regexp_partial_match.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::BOOLEAN, RegexpMatchesFunction, RegexpMatchesBind, nullptr, nullptr, RegexInitLocalState, - LogicalType::INVALID, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); ScalarFunctionSet regexp_replace("regexp_replace"); regexp_replace.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}, @@ -410,43 +410,43 @@ void RegexpFun::RegisterFunction(BuiltinFunctions &set) { RegexReplaceFunction, RegexReplaceBind, nullptr, nullptr, RegexInitLocalState)); ScalarFunctionSet regexp_extract("regexp_extract"); - regexp_extract.AddFunction( - ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::VARCHAR, RegexExtractFunction, - RegexExtractBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); - regexp_extract.AddFunction(ScalarFunction( - {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::INTEGER}, LogicalType::VARCHAR, RegexExtractFunction, - RegexExtractBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + regexp_extract.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::VARCHAR, + RegexExtractFunction, RegexExtractBind, nullptr, nullptr, + RegexInitLocalState, LogicalType::INVALID, FunctionStability::CONSISTENT, + FunctionNullHandling::SPECIAL_HANDLING)); + regexp_extract.AddFunction(ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::INTEGER}, + LogicalType::VARCHAR, RegexExtractFunction, RegexExtractBind, nullptr, + nullptr, RegexInitLocalState, LogicalType::INVALID, + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); regexp_extract.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::INTEGER, LogicalType::VARCHAR}, LogicalType::VARCHAR, RegexExtractFunction, RegexExtractBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); // REGEXP_EXTRACT(, , [[, ]...]) regexp_extract.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::LIST(LogicalType::VARCHAR)}, LogicalType::VARCHAR, RegexExtractStructFunction, RegexExtractBind, nullptr, nullptr, RegexInitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); // REGEXP_EXTRACT(, , [[, ]...], ) regexp_extract.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::LIST(LogicalType::VARCHAR), LogicalType::VARCHAR}, LogicalType::VARCHAR, RegexExtractStructFunction, RegexExtractBind, nullptr, nullptr, RegexInitLocalState, - LogicalType::INVALID, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); ScalarFunctionSet regexp_extract_all("regexp_extract_all"); regexp_extract_all.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR}, LogicalType::LIST(LogicalType::VARCHAR), RegexpExtractAll::Execute, RegexpExtractAll::Bind, nullptr, nullptr, RegexpExtractAll::InitLocalState, - LogicalType::INVALID, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); regexp_extract_all.AddFunction(ScalarFunction( {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::INTEGER}, LogicalType::LIST(LogicalType::VARCHAR), RegexpExtractAll::Execute, RegexpExtractAll::Bind, nullptr, nullptr, RegexpExtractAll::InitLocalState, - LogicalType::INVALID, FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + LogicalType::INVALID, FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); regexp_extract_all.AddFunction( ScalarFunction({LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::INTEGER, LogicalType::VARCHAR}, LogicalType::LIST(LogicalType::VARCHAR), RegexpExtractAll::Execute, RegexpExtractAll::Bind, nullptr, nullptr, RegexpExtractAll::InitLocalState, LogicalType::INVALID, - FunctionSideEffects::NO_SIDE_EFFECTS, FunctionNullHandling::SPECIAL_HANDLING)); + FunctionStability::CONSISTENT, FunctionNullHandling::SPECIAL_HANDLING)); set.AddFunction(regexp_full_match); set.AddFunction(regexp_partial_match); diff --git a/src/duckdb/src/function/scalar/system/aggregate_export.cpp b/src/duckdb/src/function/scalar/system/aggregate_export.cpp index e71255384..e1e068b23 100644 --- a/src/duckdb/src/function/scalar/system/aggregate_export.cpp +++ b/src/duckdb/src/function/scalar/system/aggregate_export.cpp @@ -171,7 +171,7 @@ static void AggregateStateCombine(DataChunk &input, ExpressionState &state_p, Ve memcpy(local_state.state_buffer0.get(), state0.GetData(), bind_data.state_size); memcpy(local_state.state_buffer1.get(), state1.GetData(), bind_data.state_size); - AggregateInputData aggr_input_data(nullptr, local_state.allocator); + AggregateInputData aggr_input_data(nullptr, local_state.allocator, AggregateCombineType::ALLOW_DESTRUCTIVE); bind_data.aggr.combine(local_state.state_vector0, local_state.state_vector1, aggr_input_data, 1); result_ptr[i] = StringVector::AddStringOrBlob(result, const_char_ptr_cast(local_state.state_buffer1.get()), @@ -213,13 +213,14 @@ static unique_ptr BindAggregateState(ClientContext &context, Scala } auto &aggr = func.Cast(); - string error; + ErrorData error; FunctionBinder function_binder(context); idx_t best_function = function_binder.BindFunction(aggr.name, aggr.functions, state_type.bound_argument_types, error); if (best_function == DConstants::INVALID_INDEX) { - throw InternalException("Could not re-bind exported aggregate %s: %s", state_type.function_name, error); + throw InternalException("Could not re-bind exported aggregate %s: %s", state_type.function_name, + error.Message()); } auto bound_aggr = aggr.functions.GetFunctionByOffset(best_function); if (bound_aggr.bind) { diff --git a/src/duckdb/src/function/scalar_function.cpp b/src/duckdb/src/function/scalar_function.cpp index ac7b80dba..ceebf3d52 100644 --- a/src/duckdb/src/function/scalar_function.cpp +++ b/src/duckdb/src/function/scalar_function.cpp @@ -8,9 +8,8 @@ FunctionLocalState::~FunctionLocalState() { ScalarFunction::ScalarFunction(string name, 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, - bind_lambda_function_t bind_lambda) + init_local_state_t init_local_state, LogicalType varargs, FunctionStability 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), @@ -20,8 +19,8 @@ ScalarFunction::ScalarFunction(string name, vector arguments, Logic 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, bind_lambda_function_t bind_lambda) + LogicalType varargs, FunctionStability side_effects, 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, bind_lambda) { } diff --git a/src/duckdb/src/function/table/arrow.cpp b/src/duckdb/src/function/table/arrow.cpp index 14044591e..d3d2aa00f 100644 --- a/src/duckdb/src/function/table/arrow.cpp +++ b/src/duckdb/src/function/table/arrow.cpp @@ -137,6 +137,23 @@ static unique_ptr GetArrowLogicalTypeNoDictionary(ArrowSchema &schema auto union_type = make_uniq(LogicalType::UNION(members)); union_type->AssignChildren(std::move(children)); return union_type; + } else if (format == "+r") { + child_list_t members; + vector> children; + idx_t n_children = idx_t(schema.n_children); + D_ASSERT(n_children == 2); + D_ASSERT(string(schema.children[0]->name) == "run_ends"); + D_ASSERT(string(schema.children[1]->name) == "values"); + for (idx_t i = 0; i < n_children; i++) { + auto type = schema.children[i]; + children.emplace_back(ArrowTableFunction::GetArrowLogicalType(*type)); + members.emplace_back(type->name, children.back()->GetDuckType()); + } + + auto struct_type = make_uniq(LogicalType::STRUCT(members)); + struct_type->AssignChildren(std::move(children)); + struct_type->SetRunEndEncoded(); + return struct_type; } else if (format == "+m") { auto &arrow_struct_type = *schema.children[0]; D_ASSERT(arrow_struct_type.n_children == 2); @@ -284,7 +301,7 @@ bool ArrowTableFunction::ArrowScanParallelStateNext(ClientContext &context, cons if (parallel_state.done) { return false; } - state.chunk_offset = 0; + state.Reset(); state.batch_index = ++parallel_state.batch_index; auto current_chunk = parallel_state.stream->GetNextChunk(); diff --git a/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp b/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp index 933c4da41..ae841179c 100644 --- a/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp +++ b/src/duckdb/src/function/table/arrow/arrow_duck_schema.cpp @@ -36,6 +36,18 @@ const ArrowType &ArrowType::GetDictionary() const { return *dictionary_type; } +void ArrowType::SetRunEndEncoded() { + D_ASSERT(children.size() == 2); + auto actual_type = children[1]->GetDuckType(); + // Override the duckdb type to the actual type + type = actual_type; + run_end_encoded = true; +} + +bool ArrowType::RunEndEncoded() const { + return run_end_encoded; +} + LogicalType ArrowType::GetDuckType(bool use_dictionary) const { if (use_dictionary && dictionary_type) { return dictionary_type->GetDuckType(); diff --git a/src/duckdb/src/function/table/arrow_conversion.cpp b/src/duckdb/src/function/table/arrow_conversion.cpp index 7aaede6f5..17ed63b0e 100644 --- a/src/duckdb/src/function/table/arrow_conversion.cpp +++ b/src/duckdb/src/function/table/arrow_conversion.cpp @@ -4,9 +4,26 @@ #include "duckdb/common/types/hugeint.hpp" #include "duckdb/common/types/arrow_aux_data.hpp" #include "duckdb/function/scalar/nested_functions.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" namespace duckdb { +namespace { + +enum class ArrowArrayPhysicalType : uint8_t { DICTIONARY_ENCODED, RUN_END_ENCODED, DEFAULT }; + +ArrowArrayPhysicalType GetArrowArrayPhysicalType(const ArrowType &type) { + if (type.HasDictionary()) { + return ArrowArrayPhysicalType::DICTIONARY_ENCODED; + } + if (type.RunEndEncoded()) { + return ArrowArrayPhysicalType::RUN_END_ENCODED; + } + return ArrowArrayPhysicalType::DEFAULT; +} + +} // namespace + static void ShiftRight(unsigned char *ar, int size, int shift) { int carry = 0; while (shift--) { @@ -18,7 +35,7 @@ static void ShiftRight(unsigned char *ar, int size, int shift) { } } -idx_t GetEffectiveOffset(ArrowArray &array, int64_t parent_offset, ArrowScanLocalState &state, +idx_t GetEffectiveOffset(ArrowArray &array, int64_t parent_offset, const ArrowScanLocalState &state, int64_t nested_offset = -1) { if (nested_offset != -1) { // The parent of this array is a list @@ -36,7 +53,7 @@ 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, +static void GetValidityMask(ValidityMask &mask, ArrowArray &array, const ArrowScanLocalState &scan_state, idx_t size, int64_t parent_offset, 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. // @@ -83,13 +100,17 @@ static void GetValidityMask(ValidityMask &mask, ArrowArray &array, ArrowScanLoca } } -static void SetValidityMask(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, idx_t size, +static void SetValidityMask(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, idx_t size, int64_t parent_offset, 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); } +static void ColumnArrowToDuckDBRunEndEncoded(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, + idx_t size, const ArrowType &arrow_type, int64_t nested_offset = -1, + ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); + static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArrayScanState &array_state, idx_t size, const ArrowType &arrow_type, int64_t nested_offset = -1, ValidityMask *parent_mask = nullptr, uint64_t parent_offset = 0); @@ -168,17 +189,27 @@ static void ArrowToDuckDBList(Vector &vector, ArrowArray &array, ArrowArrayScanS if (list_size == 0 && start_offset == 0) { D_ASSERT(!child_array.dictionary); ColumnArrowToDuckDB(child_vector, child_array, child_state, list_size, child_type, -1); - } else { - if (child_array.dictionary) { - // TODO: add support for offsets - ColumnArrowToDuckDBDictionary(child_vector, child_array, child_state, list_size, child_type, start_offset); - } else { - ColumnArrowToDuckDB(child_vector, child_array, child_state, list_size, child_type, start_offset); - } + return; + } + + auto array_physical_type = GetArrowArrayPhysicalType(child_type); + switch (array_physical_type) { + case ArrowArrayPhysicalType::DICTIONARY_ENCODED: + // TODO: add support for offsets + ColumnArrowToDuckDBDictionary(child_vector, child_array, child_state, list_size, child_type, start_offset); + break; + case ArrowArrayPhysicalType::RUN_END_ENCODED: + ColumnArrowToDuckDBRunEndEncoded(child_vector, child_array, child_state, list_size, child_type, start_offset); + break; + case ArrowArrayPhysicalType::DEFAULT: + ColumnArrowToDuckDB(child_vector, child_array, child_state, list_size, child_type, start_offset); + break; + default: + throw NotImplementedException("ArrowArrayPhysicalType not recognized"); } } -static void ArrowToDuckDBBlob(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, idx_t size, +static void ArrowToDuckDBBlob(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, idx_t size, const ArrowType &arrow_type, int64_t nested_offset, int64_t parent_offset) { auto size_type = arrow_type.GetSizeType(); SetValidityMask(vector, array, scan_state, size, parent_offset, nested_offset); @@ -263,8 +294,8 @@ 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) { +static void DirectConversion(Vector &vector, ArrowArray &array, const 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); @@ -272,8 +303,8 @@ static void DirectConversion(Vector &vector, ArrowArray &array, ArrowScanLocalSt } 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) { +static void TimeConversion(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, + int64_t nested_offset, int64_t parent_offset, 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); @@ -287,7 +318,7 @@ static void TimeConversion(Vector &vector, ArrowArray &array, ArrowScanLocalStat } } -static void TimestampTZConversion(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, +static void TimestampTZConversion(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto &validity_mask = FlatVector::Validity(vector); @@ -303,7 +334,7 @@ static void TimestampTZConversion(Vector &vector, ArrowArray &array, ArrowScanLo } } -static void IntervalConversionUs(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, +static void IntervalConversionUs(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, int64_t nested_offset, int64_t parent_offset, idx_t size, int64_t conversion) { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = @@ -317,7 +348,7 @@ static void IntervalConversionUs(Vector &vector, ArrowArray &array, ArrowScanLoc } } -static void IntervalConversionMonths(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, +static void IntervalConversionMonths(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, int64_t nested_offset, int64_t parent_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = @@ -329,7 +360,7 @@ static void IntervalConversionMonths(Vector &vector, ArrowArray &array, ArrowSca } } -static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, ArrowScanLocalState &scan_state, +static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, const ArrowScanLocalState &scan_state, int64_t nested_offset, int64_t parent_offset, idx_t size) { auto tgt_ptr = FlatVector::GetData(vector); auto src_ptr = @@ -341,6 +372,218 @@ static void IntervalConversionMonthDayNanos(Vector &vector, ArrowArray &array, A } } +template +static idx_t FindRunIndex(const RUN_END_TYPE *run_ends, idx_t count, idx_t offset) { + idx_t begin = 0; + idx_t end = count - 1; + while (begin < end) { + idx_t middle = static_cast(std::floor((begin + end) / 2)); + if (offset >= static_cast(run_ends[middle])) { + // Our offset starts after this run has ended + begin = middle + 1; + } else { + // This offset might fall into this run_end + if (middle == 0) { + return middle; + } + if (offset >= static_cast(run_ends[middle - 1])) { + // For example [0, 0, 0, 1, 1, 2] + // encoded as run_ends: [3, 5, 6] + // 3 (run_ends[0]) >= offset < 5 (run_ends[1]) + return middle; + } + end = middle - 1; + } + } + return end; +} + +template +static void FlattenRunEnds(Vector &result, ArrowRunEndEncodingState &run_end_encoding, idx_t compressed_size, + idx_t scan_offset, idx_t count) { + auto &runs = *run_end_encoding.run_ends; + auto &values = *run_end_encoding.values; + + UnifiedVectorFormat run_end_format; + UnifiedVectorFormat value_format; + runs.ToUnifiedFormat(compressed_size, run_end_format); + values.ToUnifiedFormat(compressed_size, value_format); + auto run_ends_data = run_end_format.GetData(run_end_format); + auto values_data = value_format.GetData(value_format); + auto result_data = FlatVector::GetData(result); + auto &validity = FlatVector::Validity(result); + + // According to the arrow spec, the 'run_ends' array is always valid + // so we will assume this is true and not check the validity map + + // Now construct the result vector from the run_ends and the values + + auto run = FindRunIndex(run_ends_data, compressed_size, scan_offset); + idx_t logical_index = scan_offset; + idx_t index = 0; + if (value_format.validity.AllValid()) { + // None of the compressed values are NULL + for (; run < compressed_size; run++) { + auto run_end_index = run_end_format.sel->get_index(run); + auto value_index = value_format.sel->get_index(run); + auto &value = values_data[value_index]; + auto run_end = static_cast(run_ends_data[run_end_index]); + + D_ASSERT(run_end > (logical_index + index)); + auto to_scan = run_end - (logical_index + index); + // Cap the amount to scan so we don't go over size + to_scan = MinValue(to_scan, (count - index)); + + for (idx_t i = 0; i < to_scan; i++) { + result_data[index + i] = value; + } + index += to_scan; + if (index >= count) { + if (logical_index + index >= run_end) { + // The last run was completed, forward the run index + run++; + } + break; + } + } + } else { + for (; run < compressed_size; run++) { + auto run_end_index = run_end_format.sel->get_index(run); + auto value_index = value_format.sel->get_index(run); + auto run_end = static_cast(run_ends_data[run_end_index]); + + D_ASSERT(run_end > (logical_index + index)); + auto to_scan = run_end - (logical_index + index); + // Cap the amount to scan so we don't go over size + to_scan = MinValue(to_scan, (count - index)); + + if (value_format.validity.RowIsValidUnsafe(value_index)) { + auto &value = values_data[value_index]; + for (idx_t i = 0; i < to_scan; i++) { + result_data[index + i] = value; + validity.SetValid(index + i); + } + } else { + for (idx_t i = 0; i < to_scan; i++) { + validity.SetInvalid(index + i); + } + } + index += to_scan; + if (index >= count) { + if (logical_index + index >= run_end) { + // The last run was completed, forward the run index + run++; + } + break; + } + } + } +} + +template +static void FlattenRunEndsSwitch(Vector &result, ArrowRunEndEncodingState &run_end_encoding, idx_t compressed_size, + idx_t scan_offset, idx_t size) { + auto &values = *run_end_encoding.values; + auto physical_type = values.GetType().InternalType(); + + switch (physical_type) { + case PhysicalType::INT8: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT16: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT32: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT64: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT128: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::UINT8: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::UINT16: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::UINT32: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::UINT64: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::BOOL: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::FLOAT: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::DOUBLE: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INTERVAL: + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::VARCHAR: { + // Share the string heap, we don't need to allocate new strings, we just reference the existing ones + result.SetAuxiliary(values.GetAuxiliary()); + FlattenRunEnds(result, run_end_encoding, compressed_size, scan_offset, size); + break; + } + default: + throw NotImplementedException("RunEndEncoded value type '%s' not supported yet", TypeIdToString(physical_type)); + } +} + +static void ColumnArrowToDuckDBRunEndEncoded(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) { + // Scan the 'run_ends' array + D_ASSERT(array.n_children == 2); + auto &run_ends_array = *array.children[0]; + auto &values_array = *array.children[1]; + + auto &run_ends_type = arrow_type[0]; + auto &values_type = arrow_type[1]; + D_ASSERT(vector.GetType() == values_type.GetDuckType()); + + auto &scan_state = array_state.state; + + D_ASSERT(run_ends_array.length == values_array.length); + auto compressed_size = run_ends_array.length; + // Create a vector for the run ends and the values + auto &run_end_encoding = array_state.RunEndEncoding(); + if (!run_end_encoding.run_ends) { + // The run ends and values have not been scanned yet for this array + D_ASSERT(!run_end_encoding.values); + run_end_encoding.run_ends = make_uniq(run_ends_type.GetDuckType(), compressed_size); + run_end_encoding.values = make_uniq(values_type.GetDuckType(), compressed_size); + + ColumnArrowToDuckDB(*run_end_encoding.run_ends, run_ends_array, array_state, compressed_size, run_ends_type); + auto &values = *run_end_encoding.values; + SetValidityMask(values, values_array, scan_state, compressed_size, parent_offset, nested_offset); + ColumnArrowToDuckDB(values, values_array, array_state, compressed_size, values_type); + } + + idx_t scan_offset = GetEffectiveOffset(array, parent_offset, scan_state, nested_offset); + auto physical_type = run_ends_type.GetDuckType().InternalType(); + switch (physical_type) { + case PhysicalType::INT16: + FlattenRunEndsSwitch(vector, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT32: + FlattenRunEndsSwitch(vector, run_end_encoding, compressed_size, scan_offset, size); + break; + case PhysicalType::INT64: + FlattenRunEndsSwitch(vector, run_end_encoding, compressed_size, scan_offset, size); + break; + default: + throw NotImplementedException("Type '%s' not implemented for RunEndEncoding", TypeIdToString(physical_type)); + } +} + 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) { @@ -349,6 +592,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } auto &scan_state = array_state.state; D_ASSERT(!array.dictionary); + switch (vector.GetType().id()) { case LogicalTypeId::SQLNULL: vector.Reference(Value()); @@ -389,6 +633,7 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca case LogicalTypeId::UBIGINT: case LogicalTypeId::BIGINT: case LogicalTypeId::HUGEINT: + case LogicalTypeId::UHUGEINT: case LogicalTypeId::TIMESTAMP: case LogicalTypeId::TIMESTAMP_SEC: case LogicalTypeId::TIMESTAMP_MS: @@ -614,12 +859,23 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca } } } - if (child_array.dictionary) { + + auto array_physical_type = GetArrowArrayPhysicalType(child_type); + switch (array_physical_type) { + case ArrowArrayPhysicalType::DICTIONARY_ENCODED: ColumnArrowToDuckDBDictionary(child_entry, child_array, child_state, size, child_type, nested_offset, &struct_validity_mask, array.offset); - } else { + break; + case ArrowArrayPhysicalType::RUN_END_ENCODED: + ColumnArrowToDuckDBRunEndEncoded(child_entry, child_array, child_state, size, child_type, nested_offset, + &struct_validity_mask, array.offset); + break; + case ArrowArrayPhysicalType::DEFAULT: ColumnArrowToDuckDB(child_entry, child_array, child_state, size, child_type, nested_offset, &struct_validity_mask, array.offset); + break; + default: + throw NotImplementedException("ArrowArrayPhysicalType not recognized"); } } break; @@ -639,11 +895,20 @@ static void ColumnArrowToDuckDB(Vector &vector, ArrowArray &array, ArrowArraySca auto &child_type = arrow_type[child_idx]; SetValidityMask(child, child_array, scan_state, size, parent_offset, nested_offset); + auto array_physical_type = GetArrowArrayPhysicalType(child_type); - if (child_array.dictionary) { + switch (array_physical_type) { + case ArrowArrayPhysicalType::DICTIONARY_ENCODED: ColumnArrowToDuckDBDictionary(child, child_array, child_state, size, child_type); - } else { + break; + case ArrowArrayPhysicalType::RUN_END_ENCODED: + ColumnArrowToDuckDBRunEndEncoded(child, child_array, child_state, size, child_type); + break; + case ArrowArrayPhysicalType::DEFAULT: ColumnArrowToDuckDB(child, child_array, child_state, size, child_type, nested_offset, &validity_mask); + break; + default: + throw NotImplementedException("ArrowArrayPhysicalType not recognized"); } children.push_back(std::move(child)); @@ -811,8 +1076,24 @@ static void ColumnArrowToDuckDBDictionary(Vector &vector, ArrowArray &array, Arr //! 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); - ColumnArrowToDuckDB(*base_vector, *array.dictionary, array_state, array.dictionary->length, - arrow_type.GetDictionary()); + auto &dictionary_type = arrow_type.GetDictionary(); + auto arrow_physical_type = GetArrowArrayPhysicalType(dictionary_type); + switch (arrow_physical_type) { + case ArrowArrayPhysicalType::DICTIONARY_ENCODED: + ColumnArrowToDuckDBDictionary(*base_vector, *array.dictionary, array_state, array.dictionary->length, + dictionary_type); + break; + case ArrowArrayPhysicalType::RUN_END_ENCODED: + ColumnArrowToDuckDBRunEndEncoded(*base_vector, *array.dictionary, array_state, array.dictionary->length, + dictionary_type); + break; + case ArrowArrayPhysicalType::DEFAULT: + ColumnArrowToDuckDB(*base_vector, *array.dictionary, array_state, array.dictionary->length, + dictionary_type); + break; + default: + throw NotImplementedException("ArrowArrayPhysicalType not recognized"); + }; array_state.AddDictionary(std::move(base_vector)); } auto offset_type = arrow_type.GetDuckType(); @@ -876,11 +1157,21 @@ void ArrowTableFunction::ArrowToDuckDB(ArrowScanLocalState &scan_state, const ar auto &arrow_type = *arrow_convert_data.at(col_idx); auto &array_state = scan_state.GetState(col_idx); - if (array.dictionary) { + auto array_physical_type = GetArrowArrayPhysicalType(arrow_type); + + switch (array_physical_type) { + case ArrowArrayPhysicalType::DICTIONARY_ENCODED: ColumnArrowToDuckDBDictionary(output.data[idx], array, array_state, output.size(), arrow_type); - } else { + break; + case ArrowArrayPhysicalType::RUN_END_ENCODED: + ColumnArrowToDuckDBRunEndEncoded(output.data[idx], array, array_state, output.size(), arrow_type); + break; + case ArrowArrayPhysicalType::DEFAULT: SetValidityMask(output.data[idx], array, scan_state, output.size(), parent_array.offset, -1); ColumnArrowToDuckDB(output.data[idx], array, array_state, output.size(), arrow_type); + break; + default: + throw NotImplementedException("ArrowArrayPhysicalType not recognized"); } } } diff --git a/src/duckdb/src/function/table/copy_csv.cpp b/src/duckdb/src/function/table/copy_csv.cpp index 0e2f6e7f7..ad4c540c1 100644 --- a/src/duckdb/src/function/table/copy_csv.cpp +++ b/src/duckdb/src/function/table/copy_csv.cpp @@ -1,17 +1,17 @@ #include "duckdb/common/bind_helpers.hpp" #include "duckdb/common/file_system.hpp" #include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/serializer/memory_stream.hpp" +#include "duckdb/common/serializer/write_stream.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" #include "duckdb/function/copy_function.hpp" #include "duckdb/function/scalar/string_functions.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/parser/parsed_data/copy_info.hpp" -#include "duckdb/common/serializer/write_stream.hpp" -#include "duckdb/common/serializer/memory_stream.hpp" #include @@ -85,12 +85,12 @@ void BaseCSVData::Finalize() { } } -static unique_ptr WriteCSVBind(ClientContext &context, const CopyInfo &info, const vector &names, - const vector &sql_types) { - auto bind_data = make_uniq(info.file_path, sql_types, names); +static unique_ptr WriteCSVBind(ClientContext &context, CopyFunctionBindInput &input, + const vector &names, const vector &sql_types) { + auto bind_data = make_uniq(input.info.file_path, sql_types, names); // check all the options in the copy info - for (auto &option : info.options) { + for (auto &option : input.info.options) { auto loption = StringUtil::Lower(option.first); auto &set = option.second; bind_data->options.SetWriteOption(loption, ConvertVectorToValue(set)); @@ -150,15 +150,14 @@ static unique_ptr ReadCSVBind(ClientContext &context, CopyInfo &in options.sql_types_per_column[expected_names[i]] = i; } - bind_data->FinalizeRead(context); - if (options.auto_detect) { - // 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, {&expected_types, &expected_names}); + auto buffer_manager = make_shared(context, options, bind_data->files[0], 0); + CSVSniffer sniffer(options, buffer_manager, CSVStateMachineCache::Get(context), + {&expected_types, &expected_names}); sniffer.SniffCSV(); } + bind_data->FinalizeRead(context); + return std::move(bind_data); } @@ -291,6 +290,11 @@ struct GlobalWriteCSVData : public GlobalFunctionData { handle->Write((void *)data, size); } + idx_t FileSize() { + lock_guard flock(lock); + return handle->GetFileSize(); + } + FileSystem &fs; //! The mutex for writing to the physical file mutex lock; @@ -341,6 +345,11 @@ static unique_ptr WriteCSVInitializeGlobal(ClientContext &co return std::move(global_data); } +idx_t WriteCSVFileSize(GlobalFunctionData &gstate) { + auto &global_state = gstate.Cast(); + return global_state.FileSize(); +} + static void WriteCSVChunkInternal(ClientContext &context, FunctionData &bind_data, DataChunk &cast_chunk, MemoryStream &writer, DataChunk &input, bool &written_anything) { auto &csv_data = bind_data.Cast(); @@ -519,6 +528,7 @@ void CSVCopyFunction::RegisterFunction(BuiltinFunctions &set) { info.execution_mode = WriteCSVExecutionMode; info.prepare_batch = WriteCSVPrepareBatch; info.flush_batch = WriteCSVFlushBatch; + info.file_size_bytes = WriteCSVFileSize; info.copy_from_bind = ReadCSVBind; info.copy_from_function = ReadCSVTableFunction::GetFunction(); diff --git a/src/duckdb/src/function/table/range.cpp b/src/duckdb/src/function/table/range.cpp index c99ac7e11..3e47bf111 100644 --- a/src/duckdb/src/function/table/range.cpp +++ b/src/duckdb/src/function/table/range.cpp @@ -275,6 +275,8 @@ void BuiltinFunctions::RegisterTableFunctions() { UnnestTableFunction::RegisterFunction(*this); RepeatRowTableFunction::RegisterFunction(*this); CSVSnifferFunction::RegisterFunction(*this); + ReadBlobFunction::RegisterFunction(*this); + ReadTextFunction::RegisterFunction(*this); } } // namespace duckdb diff --git a/src/duckdb/src/function/table/read_csv.cpp b/src/duckdb/src/function/table/read_csv.cpp index 715f6a4bd..47dff231b 100644 --- a/src/duckdb/src/function/table/read_csv.cpp +++ b/src/duckdb/src/function/table/read_csv.cpp @@ -1,23 +1,29 @@ #include "duckdb/function/table/read_csv.hpp" + #include "duckdb/common/enum_util.hpp" #include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/string_util.hpp" #include "duckdb/common/union_by_name.hpp" +#include "duckdb/execution/operator/csv_scanner/global_csv_state.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_error.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" #include "duckdb/execution/operator/persistent/csv_rejects_table.hpp" -#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" #include "duckdb/function/function_set.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/client_data.hpp" #include "duckdb/main/config.hpp" #include "duckdb/main/database.hpp" +#include "duckdb/main/extension_helper.hpp" #include "duckdb/parser/expression/constant_expression.hpp" #include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/parser/tableref/table_function_ref.hpp" #include "duckdb/planner/operator/logical_get.hpp" -#include "duckdb/main/extension_helper.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/base_scanner.hpp" + +#include "duckdb/execution/operator/csv_scanner/string_value_scanner.hpp" #include @@ -30,23 +36,11 @@ unique_ptr ReadCSV::OpenCSV(const string &file_path, FileCompress return CSVFileHandle::OpenFile(fs, allocator, file_path, compression); } +ReadCSVData::ReadCSVData() { +} + void ReadCSVData::FinalizeRead(ClientContext &context) { BaseCSVData::Finalize(); - // Here we identify if we can run this CSV file on parallel or not. - bool not_supported_options = options.null_padding; - - auto number_of_threads = TaskScheduler::GetScheduler(context).NumberOfThreads(); - //! If we have many csv files, we run single-threaded on each file and parallelize on the number of files - bool many_csv_files = files.size() > 1 && int64_t(files.size() * 2) >= number_of_threads; - if (options.parallel_mode != ParallelMode::PARALLEL && (many_csv_files || number_of_threads == 1)) { - single_threaded = true; - } - if (options.parallel_mode == ParallelMode::SINGLE_THREADED || not_supported_options || - options.dialect_options.new_line == NewLineIdentifier::MIX) { - // not supported for parallel CSV reading - single_threaded = true; - } - if (!options.rejects_recovery_columns.empty()) { for (auto &recovery_col : options.rejects_recovery_columns) { bool found = false; @@ -102,37 +96,39 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio "read_csv_auto or set read_csv(..., " "AUTO_DETECT=TRUE) to automatically guess columns."); } - if (options.auto_detect) { + if (options.auto_detect && !options.file_options.union_by_name) { options.file_path = result->files[0]; - // 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, {&return_types, &names}); + result->buffer_manager = make_shared(context, options, result->files[0], 0); + CSVSniffer sniffer(options, result->buffer_manager, CSVStateMachineCache::Get(context), + {&return_types, &names}); auto sniffer_result = sniffer.SniffCSV(); if (names.empty()) { names = sniffer_result.names; return_types = sniffer_result.return_types; } } - D_ASSERT(return_types.size() == names.size()); - - result->csv_types = return_types; - result->csv_names = names; + D_ASSERT(return_types.size() == names.size()); + result->options.dialect_options.num_cols = names.size(); if (options.file_options.union_by_name) { result->reader_bind = - MultiFileReader::BindUnionReader(context, return_types, names, *result, options); + MultiFileReader::BindUnionReader(context, return_types, names, *result, options); if (result->union_readers.size() > 1) { - result->column_info.emplace_back(result->csv_names, result->csv_types); + result->column_info.emplace_back(result->initial_reader->names, result->initial_reader->types); for (idx_t i = 1; i < result->union_readers.size(); i++) { - result->column_info.emplace_back(result->union_readers[i]->names, - result->union_readers[i]->return_types); + result->column_info.emplace_back(result->union_readers[i]->names, result->union_readers[i]->types); } } if (!options.sql_types_per_column.empty()) { - auto exception = BufferedCSVReader::ColumnTypesError(options.sql_types_per_column, names); - if (!exception.empty()) { - throw BinderException(exception); + auto exception = CSVError::ColumnTypesError(options.sql_types_per_column, names); + if (!exception.error_message.empty()) { + throw BinderException(exception.error_message); + } + for (idx_t i = 0; i < names.size(); i++) { + auto it = options.sql_types_per_column.find(names[i]); + if (it != options.sql_types_per_column.end()) { + return_types[i] = options.sql_type_list[it->second]; + } } } } else { @@ -140,728 +136,93 @@ static unique_ptr ReadCSVBind(ClientContext &context, TableFunctio } result->return_types = return_types; result->return_names = names; + result->csv_types = return_types; + result->csv_names = names; result->FinalizeRead(context); - return std::move(result); } //===--------------------------------------------------------------------===// -// Parallel CSV Reader CSV Global State +// Read CSV Local State //===--------------------------------------------------------------------===// - -struct ParallelCSVGlobalState : public GlobalTableFunctionState { +struct CSVLocalState : public LocalTableFunctionState { public: - ParallelCSVGlobalState(ClientContext &context, shared_ptr buffer_manager_p, - const CSVReaderOptions &options, idx_t system_threads_p, const vector &files_path_p, - bool force_parallelism_p, vector column_ids_p) - : buffer_manager(std::move(buffer_manager_p)), system_threads(system_threads_p), - force_parallelism(force_parallelism_p), column_ids(std::move(column_ids_p)), - line_info(main_mutex, batch_to_tuple_end, tuple_start, tuple_end, options.sniffer_user_mismatch_error), - sniffer_mismatch_error(options.sniffer_user_mismatch_error) { - current_file_path = files_path_p[0]; - CSVFileHandle *file_handle_ptr; - - if (!buffer_manager || options.dialect_options.skip_rows.GetValue() > 0 || - buffer_manager->file_handle->GetFilePath() != current_file_path) { - // If our buffers are too small, and we skip too many rows there is a chance things will go over-buffer - // for now don't reuse the buffer manager - buffer_manager.reset(); - file_handle = ReadCSV::OpenCSV(current_file_path, options.compression, context); - file_handle_ptr = file_handle.get(); - } else { - file_handle_ptr = buffer_manager->file_handle.get(); - } - - file_size = file_handle_ptr->FileSize(); - first_file_size = file_size; - on_disk_file = file_handle_ptr->OnDiskFile(); - bytes_read = 0; - running_threads = MaxThreads(); - - // Initialize all the book-keeping variables - auto file_count = files_path_p.size(); - line_info.current_batches.resize(file_count); - line_info.lines_read.resize(file_count); - line_info.lines_errored.resize(file_count); - tuple_start.resize(file_count); - tuple_end.resize(file_count); - tuple_end_to_batch.resize(file_count); - batch_to_tuple_end.resize(file_count); - - // Initialize the lines read - line_info.lines_read[0][0] = options.dialect_options.skip_rows.GetValue(); - if (options.dialect_options.header.GetValue()) { - line_info.lines_read[0][0]++; - } - first_position = options.dialect_options.true_start; - next_byte = options.dialect_options.true_start; - } - explicit ParallelCSVGlobalState(idx_t system_threads_p) - : system_threads(system_threads_p), line_info(main_mutex, batch_to_tuple_end, tuple_start, tuple_end, "") { - running_threads = MaxThreads(); - } - - ~ParallelCSVGlobalState() override { - } - - //! How many bytes were read up to this point - atomic bytes_read; - //! Size of current file - idx_t file_size; - -public: - idx_t MaxThreads() const override; - //! Updates the CSV reader with the next buffer to read. Returns false if no more buffers are available. - bool Next(ClientContext &context, const ReadCSVData &bind_data, unique_ptr &reader); - //! Verify if the CSV File was read correctly - void Verify(); - - void UpdateVerification(VerificationPositions positions, idx_t file_number, idx_t batch_idx); - - void UpdateLinesRead(CSVBufferRead &buffer_read, idx_t file_idx); - - void DecrementThread(); - - bool Finished(); - - double GetProgress(const ReadCSVData &bind_data) const { - idx_t total_files = bind_data.files.size(); - - // get the progress WITHIN the current file - double progress; - if (file_size == 0) { - progress = 1.0; - } else { - progress = double(bytes_read) / double(file_size); - } - // now get the total percentage of files read - double percentage = double(file_index - 1) / total_files; - percentage += (double(1) / double(total_files)) * progress; - return percentage * 100; + explicit CSVLocalState(unique_ptr csv_reader_p) : csv_reader(std::move(csv_reader_p)) { } -private: - //! File Handle for current file - shared_ptr buffer_manager; - - //! The index of the next file to read (i.e. current file + 1) - idx_t file_index = 1; - string current_file_path; - - //! Mutex to lock when getting next batch of bytes (Parallel Only) - mutex main_mutex; - //! Byte set from for last thread - idx_t next_byte = 0; - //! Size of first file - idx_t first_file_size = 0; - //! Whether or not this is an on-disk file - bool on_disk_file = true; - //! Basically max number of threads in DuckDB - idx_t system_threads; - //! Current batch index - idx_t batch_index = 0; - idx_t local_batch_index = 0; - - //! Forces parallelism for small CSV Files, should only be used for testing. - bool force_parallelism = false; - //! First Position of First Buffer - idx_t first_position = 0; - //! Current File Number - idx_t max_tuple_end = 0; - //! The vector stores positions where threads ended the last line they read in the CSV File, and the set stores - //! Positions where they started reading the first line. - vector> tuple_end; - vector> tuple_start; - //! Tuple end to batch - vector> tuple_end_to_batch; - //! Batch to Tuple End - vector> batch_to_tuple_end; - idx_t running_threads = 0; - //! The column ids to read - vector column_ids; - //! Line Info used in error messages - LineInfo line_info; - //! Current Buffer index - idx_t cur_buffer_idx = 0; - //! Only used if we don't run auto_detection first - unique_ptr file_handle; - - string sniffer_mismatch_error; + //! The CSV reader + unique_ptr csv_reader; + bool done = false; }; -idx_t ParallelCSVGlobalState::MaxThreads() const { - if (force_parallelism || !on_disk_file) { - return system_threads; - } - idx_t one_mb = 1000000; // We initialize max one thread per Mb - idx_t threads_per_mb = first_file_size / one_mb + 1; - if (threads_per_mb < system_threads || threads_per_mb == 1) { - return threads_per_mb; - } - - return system_threads; -} - -void ParallelCSVGlobalState::DecrementThread() { - lock_guard parallel_lock(main_mutex); - D_ASSERT(running_threads > 0); - running_threads--; -} - -bool ParallelCSVGlobalState::Finished() { - lock_guard parallel_lock(main_mutex); - return running_threads == 0; -} - -void ParallelCSVGlobalState::Verify() { - // All threads are done, we run some magic sweet verification code - lock_guard parallel_lock(main_mutex); - if (running_threads == 0) { - D_ASSERT(tuple_end.size() == tuple_start.size()); - for (idx_t i = 0; i < tuple_start.size(); i++) { - auto ¤t_tuple_end = tuple_end[i]; - auto ¤t_tuple_start = tuple_start[i]; - // figure out max value of last_pos - if (current_tuple_end.empty()) { - return; - } - auto max_value = *max_element(std::begin(current_tuple_end), std::end(current_tuple_end)); - for (idx_t tpl_idx = 0; tpl_idx < current_tuple_end.size(); tpl_idx++) { - auto last_pos = current_tuple_end[tpl_idx]; - auto first_pos = current_tuple_start.find(last_pos); - if (first_pos == current_tuple_start.end()) { - // this might be necessary due to carriage returns outside buffer scopes. - first_pos = current_tuple_start.find(last_pos + 1); - } - if (first_pos == current_tuple_start.end() && last_pos != max_value) { - auto batch_idx = tuple_end_to_batch[i][last_pos]; - auto problematic_line = line_info.GetLine(batch_idx); - throw InvalidInputException( - "CSV File not supported for multithreading. This can be a problematic line in your CSV File or " - "that this CSV can't be read in Parallel. Please, inspect if the line %llu is correct. If so, " - "please run single-threaded CSV Reading by setting parallel=false in the read_csv call. %s", - problematic_line, sniffer_mismatch_error); - } - } - } - } -} - -void LineInfo::Verify(idx_t file_idx, idx_t batch_idx, idx_t cur_first_pos) { - auto &tuple_start_set = tuple_start[file_idx]; - auto &processed_batches = batch_to_tuple_end[file_idx]; - auto &tuple_end_vec = tuple_end[file_idx]; - bool has_error = false; - idx_t problematic_line; - if (batch_idx == 0 || tuple_start_set.empty()) { - return; - } - for (idx_t cur_batch = 0; cur_batch < batch_idx - 1; cur_batch++) { - auto cur_end = tuple_end_vec[processed_batches[cur_batch]]; - auto first_pos = tuple_start_set.find(cur_end); - if (first_pos == tuple_start_set.end()) { - has_error = true; - problematic_line = GetLine(cur_batch); - break; - } - } - if (!has_error) { - auto cur_end = tuple_end_vec[processed_batches[batch_idx - 1]]; - if (cur_end != cur_first_pos) { - has_error = true; - problematic_line = GetLine(batch_idx); - } - } - if (has_error) { - throw InvalidInputException( - "CSV File not supported for multithreading. This can be a problematic line in your CSV File or " - "that this CSV can't be read in Parallel. Please, inspect if the line %llu is correct. If so, " - "please run single-threaded CSV Reading by setting parallel=false in the read_csv call.\n %s", - problematic_line, sniffer_mismatch_error); - } -} -bool ParallelCSVGlobalState::Next(ClientContext &context, const ReadCSVData &bind_data, - unique_ptr &reader) { - lock_guard parallel_lock(main_mutex); - if (!buffer_manager && file_handle) { - buffer_manager = make_shared(context, std::move(file_handle), bind_data.options); - } - if (!buffer_manager) { - return false; - } - auto current_buffer = buffer_manager->GetBuffer(cur_buffer_idx); - auto next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); - - if (!current_buffer) { - // This means we are done with the current file, we need to go to the next one (if exists). - if (file_index < bind_data.files.size()) { - current_file_path = bind_data.files[file_index]; - file_handle = ReadCSV::OpenCSV(current_file_path, bind_data.options.compression, context); - buffer_manager = - make_shared(context, std::move(file_handle), bind_data.options, file_index); - cur_buffer_idx = 0; - first_position = 0; - local_batch_index = 0; - - line_info.lines_read[file_index++][local_batch_index] = - bind_data.options.dialect_options.header.GetValue() ? 1 : 0; - - current_buffer = buffer_manager->GetBuffer(cur_buffer_idx); - next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); - } else { - // We are done scanning. - reader.reset(); - return false; - } - } - // set up the current buffer - line_info.current_batches[file_index - 1].insert(local_batch_index); - idx_t bytes_per_local_state = current_buffer->actual_size / MaxThreads() + 1; - auto result = make_uniq( - buffer_manager->GetBuffer(cur_buffer_idx), buffer_manager->GetBuffer(cur_buffer_idx + 1), next_byte, - next_byte + bytes_per_local_state, batch_index++, local_batch_index++, &line_info); - // move the byte index of the CSV reader to the next buffer - next_byte += bytes_per_local_state; - if (next_byte >= current_buffer->actual_size) { - // We replace the current buffer with the next buffer - next_byte = 0; - bytes_read += current_buffer->actual_size; - current_buffer = std::move(next_buffer); - cur_buffer_idx++; - if (current_buffer) { - // Next buffer gets the next-next buffer - next_buffer = buffer_manager->GetBuffer(cur_buffer_idx + 1); - } - } - if (!reader || reader->options.file_path != current_file_path) { - // we either don't have a reader, or the reader was created for a different file - // we need to create a new reader and instantiate it - if (file_index > 0 && file_index <= bind_data.union_readers.size() && bind_data.union_readers[file_index - 1]) { - // we are doing UNION BY NAME - fetch the options from the union reader for this file - auto &union_reader = *bind_data.union_readers[file_index - 1]; - reader = make_uniq(context, union_reader.options, std::move(result), first_position, - union_reader.GetTypes(), file_index - 1); - reader->names = union_reader.GetNames(); - } else if (file_index <= bind_data.column_info.size()) { - // Serialized Union By name - reader = make_uniq(context, bind_data.options, std::move(result), first_position, - bind_data.column_info[file_index - 1].types, file_index - 1); - reader->names = bind_data.column_info[file_index - 1].names; - } else { - // regular file - use the standard options - if (!result) { - return false; - } - reader = make_uniq(context, bind_data.options, std::move(result), first_position, - bind_data.csv_types, file_index - 1); - reader->names = bind_data.csv_names; - } - reader->options.file_path = current_file_path; - MultiFileReader::InitializeReader(*reader, bind_data.options.file_options, bind_data.reader_bind, - bind_data.return_types, bind_data.return_names, column_ids, nullptr, - bind_data.files.front(), context); - } else { - // update the current reader - reader->SetBufferRead(std::move(result)); - } - - return true; -} -void ParallelCSVGlobalState::UpdateVerification(VerificationPositions positions, idx_t file_number_p, idx_t batch_idx) { - lock_guard parallel_lock(main_mutex); - if (positions.end_of_last_line > max_tuple_end) { - max_tuple_end = positions.end_of_last_line; - } - tuple_end_to_batch[file_number_p][positions.end_of_last_line] = batch_idx; - batch_to_tuple_end[file_number_p][batch_idx] = tuple_end[file_number_p].size(); - tuple_start[file_number_p].insert(positions.beginning_of_first_line); - tuple_end[file_number_p].push_back(positions.end_of_last_line); -} - -void ParallelCSVGlobalState::UpdateLinesRead(CSVBufferRead &buffer_read, idx_t file_idx) { - auto batch_idx = buffer_read.local_batch_index; - auto lines_read = buffer_read.lines_read; - lock_guard parallel_lock(main_mutex); - line_info.current_batches[file_idx].erase(batch_idx); - line_info.lines_read[file_idx][batch_idx] += lines_read; -} - -bool LineInfo::CanItGetLine(idx_t file_idx, idx_t batch_idx) { - lock_guard parallel_lock(main_mutex); - if (current_batches.empty() || done) { - return true; - } - if (file_idx >= current_batches.size() || current_batches[file_idx].empty()) { - return true; - } - auto min_value = *current_batches[file_idx].begin(); - if (min_value >= batch_idx) { - return true; - } - return false; -} - -void LineInfo::Increment(idx_t file_idx, idx_t batch_idx) { - auto parallel_lock = duckdb::make_uniq>(main_mutex); - lines_errored[file_idx][batch_idx]++; -} - -// Returns the 1-indexed line number -idx_t LineInfo::GetLine(idx_t batch_idx, idx_t line_error, idx_t file_idx, idx_t cur_start, bool verify, - bool stop_at_first) { - unique_ptr> parallel_lock; - if (!verify) { - parallel_lock = duckdb::make_uniq>(main_mutex); - } - idx_t line_count = 0; - - if (!stop_at_first) { - // Figure out the amount of lines read in the current file - for (idx_t cur_batch_idx = 0; cur_batch_idx <= batch_idx; cur_batch_idx++) { - if (cur_batch_idx < batch_idx) { - line_count += lines_errored[file_idx][cur_batch_idx]; - } - line_count += lines_read[file_idx][cur_batch_idx]; - } - return line_count + line_error + 1; - } - - // Otherwise, check if we already have an error on another thread - if (done) { - // line count is 0-indexed, but we want to return 1-indexed - return first_line + 1; - } - for (idx_t i = 0; i <= batch_idx; i++) { - if (lines_read[file_idx].find(i) == lines_read[file_idx].end() && i != batch_idx) { - throw InternalException("Missing batch index on Parallel CSV Reader GetLine"); - } - line_count += lines_read[file_idx][i]; - } +//===--------------------------------------------------------------------===// +// Read CSV Functions +//===--------------------------------------------------------------------===// +static unique_ptr ReadCSVInitGlobal(ClientContext &context, TableFunctionInitInput &input) { + auto &bind_data = input.bind_data->Cast(); - // before we are done, if this is not a call in Verify() we must check Verify up to this batch - if (!verify) { - Verify(file_idx, batch_idx, cur_start); + // Create the temporary rejects table + auto rejects_table = bind_data.options.rejects_table_name; + if (!rejects_table.empty()) { + CSVRejectsTable::GetOrCreate(context, rejects_table)->InitializeTable(context, bind_data); } - done = true; - first_line = line_count + line_error; - // line count is 0-indexed, but we want to return 1-indexed - return first_line + 1; -} - -static unique_ptr ParallelCSVInitGlobal(ClientContext &context, - TableFunctionInitInput &input) { - auto &bind_data = input.bind_data->CastNoConst(); if (bind_data.files.empty()) { // This can happen when a filename based filter pushdown has eliminated all possible files for this scan. - return make_uniq(context.db->NumberOfThreads()); + return nullptr; } - bind_data.options.file_path = bind_data.files[0]; - auto buffer_manager = bind_data.buffer_manager; - return make_uniq(context, buffer_manager, bind_data.options, context.db->NumberOfThreads(), - bind_data.files, ClientConfig::GetConfig(context).verify_parallelism, - input.column_ids); + return make_uniq(context, bind_data.buffer_manager, bind_data.options, + context.db->NumberOfThreads(), bind_data.files, input.column_ids, bind_data); } -//===--------------------------------------------------------------------===// -// Read CSV Local State -//===--------------------------------------------------------------------===// -struct ParallelCSVLocalState : public LocalTableFunctionState { -public: - explicit ParallelCSVLocalState(unique_ptr csv_reader_p) : csv_reader(std::move(csv_reader_p)) { +unique_ptr ReadCSVInitLocal(ExecutionContext &context, TableFunctionInitInput &input, + GlobalTableFunctionState *global_state_p) { + if (!global_state_p) { + return nullptr; } - - //! The CSV reader - unique_ptr csv_reader; - CSVBufferRead previous_buffer; - bool done = false; -}; - -unique_ptr ParallelReadCSVInitLocal(ExecutionContext &context, TableFunctionInitInput &input, - GlobalTableFunctionState *global_state_p) { - auto &csv_data = input.bind_data->Cast(); - auto &global_state = global_state_p->Cast(); - unique_ptr csv_reader; - auto has_next = global_state.Next(context.client, csv_data, csv_reader); - if (!has_next) { + auto &global_state = global_state_p->Cast(); + auto csv_scanner = global_state.Next(); + if (!csv_scanner) { global_state.DecrementThread(); - csv_reader.reset(); } - return make_uniq(std::move(csv_reader)); + return make_uniq(std::move(csv_scanner)); } -static void ParallelReadCSVFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { +static void ReadCSVFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { auto &bind_data = data_p.bind_data->Cast(); - auto &csv_global_state = data_p.global_state->Cast(); - auto &csv_local_state = data_p.local_state->Cast(); + if (!data_p.global_state) { + return; + } + auto &csv_global_state = data_p.global_state->Cast(); + auto &csv_local_state = data_p.local_state->Cast(); if (!csv_local_state.csv_reader) { // no csv_reader was set, this can happen when a filename-based filter has filtered out all possible files return; } - do { if (output.size() != 0) { - MultiFileReader::FinalizeChunk(bind_data.reader_bind, csv_local_state.csv_reader->reader_data, output); + MultiFileReader::FinalizeChunk(bind_data.reader_bind, + csv_local_state.csv_reader->csv_file_scan->reader_data, output); break; } - if (csv_local_state.csv_reader->finished) { - auto verification_updates = csv_local_state.csv_reader->GetVerificationPositions(); - csv_global_state.UpdateVerification(verification_updates, - csv_local_state.csv_reader->buffer->buffer->file_idx, - csv_local_state.csv_reader->buffer->local_batch_index); - csv_global_state.UpdateLinesRead(*csv_local_state.csv_reader->buffer, csv_local_state.csv_reader->file_idx); - auto has_next = csv_global_state.Next(context, bind_data, csv_local_state.csv_reader); - if (csv_local_state.csv_reader) { - csv_local_state.csv_reader->linenr = 0; - } - if (!has_next) { + if (csv_local_state.csv_reader->FinishedIterator()) { + csv_local_state.csv_reader->csv_file_scan->error_handler->Insert( + csv_local_state.csv_reader->GetBoundaryIndex(), csv_local_state.csv_reader->GetLinesRead()); + csv_local_state.csv_reader = csv_global_state.Next(); + if (!csv_local_state.csv_reader) { csv_global_state.DecrementThread(); break; } } - csv_local_state.csv_reader->ParseCSV(output); + csv_local_state.csv_reader->Flush(output); } while (true); - if (csv_global_state.Finished()) { - csv_global_state.Verify(); - } -} - -//===--------------------------------------------------------------------===// -// Single-Threaded CSV Reader -//===--------------------------------------------------------------------===// -struct SingleThreadedCSVState : public GlobalTableFunctionState { - explicit SingleThreadedCSVState(idx_t total_files) : total_files(total_files), next_file(0), progress_in_files(0) { - } - - mutex csv_lock; - unique_ptr initial_reader; - //! The total number of files to read from - idx_t total_files; - //! The index of the next file to read (i.e. current file + 1) - atomic next_file; - //! How far along we are in reading the current set of open files - //! This goes from [0...next_file] * 100 - atomic progress_in_files; - //! The set of SQL types - vector csv_types; - //! The set of SQL names to be read from the file - vector csv_names; - //! The column ids to read - vector column_ids; - - idx_t MaxThreads() const override { - return total_files; - } - - double GetProgress(const ReadCSVData &bind_data) const { - D_ASSERT(total_files == bind_data.files.size()); - D_ASSERT(progress_in_files <= total_files * 100); - return (double(progress_in_files) / double(total_files)); - } - - unique_ptr GetCSVReader(ClientContext &context, ReadCSVData &bind_data, idx_t &file_index, - idx_t &total_size) { - return GetCSVReaderInternal(context, bind_data, file_index, total_size); - } - -private: - unique_ptr GetCSVReaderInternal(ClientContext &context, ReadCSVData &bind_data, - idx_t &file_index, idx_t &total_size) { - CSVReaderOptions options; - { - lock_guard l(csv_lock); - if (initial_reader) { - total_size = initial_reader->file_handle ? initial_reader->file_handle->FileSize() : 0; - return std::move(initial_reader); - } - if (next_file >= total_files) { - return nullptr; - } - options = bind_data.options; - file_index = next_file; - next_file++; - } - // reuse csv_readers was created during binding - unique_ptr result; - if (file_index < bind_data.union_readers.size() && bind_data.union_readers[file_index]) { - result = std::move(bind_data.union_readers[file_index]); - } else { - auto union_by_name = options.file_options.union_by_name; - options.file_path = bind_data.files[file_index]; - result = make_uniq(context, std::move(options), csv_types); - if (!union_by_name) { - result->names = csv_names; - } - MultiFileReader::InitializeReader(*result, bind_data.options.file_options, bind_data.reader_bind, - bind_data.return_types, bind_data.return_names, column_ids, nullptr, - bind_data.files.front(), context); - } - total_size = result->file_handle->FileSize(); - return result; - } -}; - -struct SingleThreadedCSVLocalState : public LocalTableFunctionState { -public: - explicit SingleThreadedCSVLocalState() : bytes_read(0), total_size(0), current_progress(0), file_index(0) { - } - - //! The CSV reader - unique_ptr csv_reader; - //! The current amount of bytes read by this reader - idx_t bytes_read; - //! The total amount of bytes in the file - idx_t total_size; - //! The current progress from 0..100 - idx_t current_progress; - //! The file index of this reader - idx_t file_index; -}; - -static unique_ptr SingleThreadedCSVInit(ClientContext &context, - TableFunctionInitInput &input) { - auto &bind_data = input.bind_data->CastNoConst(); - auto result = make_uniq(bind_data.files.size()); - if (bind_data.files.empty()) { - // This can happen when a filename based filter pushdown has eliminated all possible files for this scan. - return std::move(result); - } else { - bind_data.options.file_path = bind_data.files[0]; - result->initial_reader = make_uniq(context, bind_data.options, bind_data.csv_types); - if (!bind_data.options.file_options.union_by_name) { - result->initial_reader->names = bind_data.csv_names; - } - if (bind_data.options.auto_detect) { - bind_data.options = result->initial_reader->options; - } - } - MultiFileReader::InitializeReader(*result->initial_reader, bind_data.options.file_options, bind_data.reader_bind, - bind_data.return_types, bind_data.return_names, input.column_ids, input.filters, - bind_data.files.front(), context); - for (auto &reader : bind_data.union_readers) { - if (!reader) { - continue; - } - MultiFileReader::InitializeReader(*reader, bind_data.options.file_options, bind_data.reader_bind, - bind_data.return_types, bind_data.return_names, input.column_ids, - input.filters, bind_data.files.front(), context); - } - result->column_ids = input.column_ids; - - if (!bind_data.options.file_options.union_by_name) { - // if we are reading multiple files - run auto-detect only on the first file - // UNLESS union by name is turned on - in that case we assume that different files have different schemas - // as such, we need to re-run the auto detection on each file - bind_data.options.auto_detect = false; - } - result->csv_types = bind_data.csv_types; - result->csv_names = bind_data.csv_names; - result->next_file = 1; - return std::move(result); -} - -unique_ptr SingleThreadedReadCSVInitLocal(ExecutionContext &context, - TableFunctionInitInput &input, - GlobalTableFunctionState *global_state_p) { - auto &bind_data = input.bind_data->CastNoConst(); - auto &data = global_state_p->Cast(); - auto result = make_uniq(); - result->csv_reader = data.GetCSVReader(context.client, bind_data, result->file_index, result->total_size); - return std::move(result); -} - -static void SingleThreadedCSVFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { - auto &bind_data = data_p.bind_data->CastNoConst(); - auto &data = data_p.global_state->Cast(); - auto &lstate = data_p.local_state->Cast(); - if (!lstate.csv_reader) { - // no csv_reader was set, this can happen when a filename-based filter has filtered out all possible files - return; - } - - do { - lstate.csv_reader->ParseCSV(output); - // update the number of bytes read - D_ASSERT(lstate.bytes_read <= lstate.csv_reader->bytes_in_chunk); - auto bytes_read = MinValue(lstate.total_size, lstate.csv_reader->bytes_in_chunk); - auto current_progress = lstate.total_size == 0 ? 100 : 100 * bytes_read / lstate.total_size; - if (current_progress > lstate.current_progress) { - if (current_progress > 100) { - throw InternalException("Progress should never exceed 100"); - } - data.progress_in_files += current_progress - lstate.current_progress; - lstate.current_progress = current_progress; - } - if (output.size() == 0) { - // exhausted this file, but we might have more files we can read - auto csv_reader = data.GetCSVReader(context, bind_data, lstate.file_index, lstate.total_size); - // add any left-over progress for this file to the progress bar - if (lstate.current_progress < 100) { - data.progress_in_files += 100 - lstate.current_progress; - } - // reset the current progress - lstate.current_progress = 0; - lstate.bytes_read = 0; - lstate.csv_reader = std::move(csv_reader); - if (!lstate.csv_reader) { - // no more files - we are done - return; - } - lstate.bytes_read = 0; - } else { - MultiFileReader::FinalizeChunk(bind_data.reader_bind, lstate.csv_reader->reader_data, output); - break; - } - } while (true); -} - -//===--------------------------------------------------------------------===// -// Read CSV Functions -//===--------------------------------------------------------------------===// -static unique_ptr ReadCSVInitGlobal(ClientContext &context, TableFunctionInitInput &input) { - auto &bind_data = input.bind_data->Cast(); - - // Create the temporary rejects table - auto rejects_table = bind_data.options.rejects_table_name; - if (!rejects_table.empty()) { - CSVRejectsTable::GetOrCreate(context, rejects_table)->InitializeTable(context, bind_data); - } - if (bind_data.single_threaded) { - return SingleThreadedCSVInit(context, input); - } else { - return ParallelCSVInitGlobal(context, input); - } -} - -unique_ptr ReadCSVInitLocal(ExecutionContext &context, TableFunctionInitInput &input, - GlobalTableFunctionState *global_state_p) { - auto &csv_data = input.bind_data->Cast(); - if (csv_data.single_threaded) { - return SingleThreadedReadCSVInitLocal(context, input, global_state_p); - } else { - return ParallelReadCSVInitLocal(context, input, global_state_p); - } -} - -static void ReadCSVFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { - auto &bind_data = data_p.bind_data->Cast(); - if (bind_data.single_threaded) { - SingleThreadedCSVFunction(context, data_p, output); - } else { - ParallelReadCSVFunction(context, data_p, output); - } } static idx_t CSVReaderGetBatchIndex(ClientContext &context, const FunctionData *bind_data_p, LocalTableFunctionState *local_state, GlobalTableFunctionState *global_state) { - auto &bind_data = bind_data_p->Cast(); - if (bind_data.single_threaded) { - auto &data = local_state->Cast(); - return data.file_index; - } - auto &data = local_state->Cast(); - return data.csv_reader->buffer->batch_index; + auto &data = local_state->Cast(); + return data.csv_reader->scanner_idx; } void ReadCSVTableFunction::ReadCSVAddNamedParameters(TableFunction &table_function) { @@ -898,19 +259,15 @@ void ReadCSVTableFunction::ReadCSVAddNamedParameters(TableFunction &table_functi table_function.named_parameters["types"] = LogicalType::ANY; table_function.named_parameters["names"] = LogicalType::LIST(LogicalType::VARCHAR); table_function.named_parameters["column_names"] = LogicalType::LIST(LogicalType::VARCHAR); + table_function.named_parameters["parallel"] = LogicalType::BOOLEAN; MultiFileReader::AddParameters(table_function); } double CSVReaderProgress(ClientContext &context, const FunctionData *bind_data_p, const GlobalTableFunctionState *global_state) { auto &bind_data = bind_data_p->Cast(); - if (bind_data.single_threaded) { - auto &data = global_state->Cast(); - return data.GetProgress(bind_data); - } else { - auto &data = global_state->Cast(); - return data.GetProgress(bind_data); - } + auto &data = global_state->Cast(); + return data.GetProgress(bind_data); } void CSVComplexFilterPushdown(ClientContext &context, LogicalGet &get, FunctionData *bind_data_p, diff --git a/src/duckdb/src/function/table/read_file.cpp b/src/duckdb/src/function/table/read_file.cpp new file mode 100644 index 000000000..f7f567ca4 --- /dev/null +++ b/src/duckdb/src/function/table/read_file.cpp @@ -0,0 +1,242 @@ +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/function/table/range.hpp" +#include "utf8proc_wrapper.hpp" + +namespace duckdb { + +struct ReadBlobOperation { + static constexpr const char *NAME = "read_blob"; + static constexpr const char *FILE_TYPE = "blob"; + + static inline LogicalType TYPE() { + return LogicalType::BLOB; + } + + static inline void VERIFY(const string &, const string_t &) { + } +}; + +struct ReadTextOperation { + static constexpr const char *NAME = "read_text"; + static constexpr const char *FILE_TYPE = "text"; + + static inline LogicalType TYPE() { + return LogicalType::VARCHAR; + } + + static inline void VERIFY(const string &filename, const string_t &content) { + if (Utf8Proc::Analyze(content.GetData(), content.GetSize()) == UnicodeType::INVALID) { + throw InvalidInputException( + "read_text: could not read content of file '%s' as valid UTF-8 encoded text. You " + "may want to use read_blob instead.", + filename); + } + } +}; + +//------------------------------------------------------------------------------ +// Bind +//------------------------------------------------------------------------------ +struct ReadFileBindData : public TableFunctionData { + vector files; + + static constexpr const idx_t FILE_NAME_COLUMN = 0; + static constexpr const idx_t FILE_CONTENT_COLUMN = 1; + static constexpr const idx_t FILE_SIZE_COLUMN = 2; + static constexpr const idx_t FILE_LAST_MODIFIED_COLUMN = 3; +}; + +template +static unique_ptr ReadFileBind(ClientContext &context, TableFunctionBindInput &input, + vector &return_types, vector &names) { + auto result = make_uniq(); + result->files = MultiFileReader::GetFileList(context, input.inputs[0], OP::FILE_TYPE, FileGlobOptions::ALLOW_EMPTY); + + return_types.push_back(LogicalType::VARCHAR); + names.push_back("filename"); + return_types.push_back(OP::TYPE()); + names.push_back("content"); + return_types.push_back(LogicalType::BIGINT); + names.push_back("size"); + return_types.push_back(LogicalType::TIMESTAMP); + names.push_back("last_modified"); + + return std::move(result); +} + +//------------------------------------------------------------------------------ +// Global state +//------------------------------------------------------------------------------ +struct ReadFileGlobalState : public GlobalTableFunctionState { + ReadFileGlobalState() : current_file_idx(0) { + } + + idx_t current_file_idx; + vector files; + vector column_ids; + bool requires_file_open = false; +}; + +static unique_ptr ReadFileInitGlobal(ClientContext &context, TableFunctionInitInput &input) { + auto &bind_data = input.bind_data->Cast(); + auto result = make_uniq(); + + result->files = bind_data.files; + result->current_file_idx = 0; + result->column_ids = input.column_ids; + + for (const auto &column_id : input.column_ids) { + // For everything except the 'file' name column, we need to open the file + if (column_id != ReadFileBindData::FILE_NAME_COLUMN && column_id != COLUMN_IDENTIFIER_ROW_ID) { + result->requires_file_open = true; + break; + } + } + + return std::move(result); +} + +//------------------------------------------------------------------------------ +// Execute +//------------------------------------------------------------------------------ +static void AssertMaxFileSize(const string &file_name, idx_t file_size) { + const auto max_file_size = NumericLimits::Maximum(); + if (file_size > max_file_size) { + auto max_byte_size_format = StringUtil::BytesToHumanReadableString(max_file_size); + auto file_byte_size_format = StringUtil::BytesToHumanReadableString(file_size); + auto error_msg = StringUtil::Format("File '%s' size (%s) exceeds maximum allowed file (%s)", file_name.c_str(), + file_byte_size_format, max_byte_size_format); + throw InvalidInputException(error_msg); + } +} + +template +static void ReadFileExecute(ClientContext &context, TableFunctionInput &input, DataChunk &output) { + auto &bind_data = input.bind_data->Cast(); + auto &state = input.global_state->Cast(); + auto &fs = FileSystem::GetFileSystem(context); + + auto output_count = MinValue(STANDARD_VECTOR_SIZE, bind_data.files.size() - state.current_file_idx); + + // We utilize projection pushdown here to only read the file content if the 'data' column is requested + for (idx_t out_idx = 0; out_idx < output_count; out_idx++) { + // Add the file name to the output + auto &file_name = bind_data.files[state.current_file_idx + out_idx]; + + unique_ptr file_handle = nullptr; + + // Given the columns requested, do we even need to open the file? + if (state.requires_file_open) { + file_handle = fs.OpenFile(file_name, FileFlags::FILE_FLAGS_READ); + } + + for (idx_t col_idx = 0; col_idx < state.column_ids.size(); col_idx++) { + // We utilize projection pushdown to avoid potentially expensive fs operations. + auto proj_idx = state.column_ids[col_idx]; + if (proj_idx == COLUMN_IDENTIFIER_ROW_ID) { + continue; + } + try { + switch (proj_idx) { + case ReadFileBindData::FILE_NAME_COLUMN: { + auto &file_name_vector = output.data[col_idx]; + auto file_name_string = StringVector::AddString(file_name_vector, file_name); + FlatVector::GetData(file_name_vector)[out_idx] = file_name_string; + } break; + case ReadFileBindData::FILE_CONTENT_COLUMN: { + auto file_size = file_handle->GetFileSize(); + AssertMaxFileSize(file_name, file_size); + auto &file_content_vector = output.data[col_idx]; + auto content_string = StringVector::EmptyString(file_content_vector, file_size); + file_handle->Read(content_string.GetDataWriteable(), file_size); + content_string.Finalize(); + + OP::VERIFY(file_name, content_string); + + FlatVector::GetData(file_content_vector)[out_idx] = content_string; + } break; + case ReadFileBindData::FILE_SIZE_COLUMN: { + auto &file_size_vector = output.data[col_idx]; + FlatVector::GetData(file_size_vector)[out_idx] = file_handle->GetFileSize(); + } break; + case ReadFileBindData::FILE_LAST_MODIFIED_COLUMN: { + auto &last_modified_vector = output.data[col_idx]; + // This can sometimes fail (e.g. httpfs file system cant always parse the last modified time + // correctly) + try { + auto timestamp_seconds = Timestamp::FromEpochSeconds(fs.GetLastModifiedTime(*file_handle)); + FlatVector::GetData(last_modified_vector)[out_idx] = timestamp_seconds; + } catch (std::exception &ex) { + ErrorData error(ex); + if (error.Type() == ExceptionType::CONVERSION) { + FlatVector::SetNull(last_modified_vector, out_idx, true); + } else { + throw; + } + } + } break; + default: + throw InternalException("Unsupported column index for read_file"); + } + } + // Filesystems are not required to support all operations, so we just set the column to NULL if not + // implemented + catch (std::exception &ex) { + ErrorData error(ex); + if (error.Type() == ExceptionType::NOT_IMPLEMENTED) { + FlatVector::SetNull(output.data[col_idx], out_idx, true); + } else { + throw; + } + } + } + } + + state.current_file_idx += output_count; + output.SetCardinality(output_count); +} + +//------------------------------------------------------------------------------ +// Misc +//------------------------------------------------------------------------------ + +static double ReadFileProgress(ClientContext &context, const FunctionData *bind_data, + const GlobalTableFunctionState *gstate) { + auto &state = gstate->Cast(); + return static_cast(state.current_file_idx) / static_cast(state.files.size()); +} + +static unique_ptr ReadFileCardinality(ClientContext &context, const FunctionData *bind_data_p) { + auto &bind_data = bind_data_p->Cast(); + auto result = make_uniq(); + result->has_max_cardinality = true; + result->max_cardinality = bind_data.files.size(); + result->has_estimated_cardinality = true; + result->estimated_cardinality = bind_data.files.size(); + return result; +} + +//------------------------------------------------------------------------------ +// Register +//------------------------------------------------------------------------------ +template +static TableFunction GetFunction() { + TableFunction func(OP::NAME, {LogicalType::VARCHAR}, ReadFileExecute, ReadFileBind, ReadFileInitGlobal); + func.table_scan_progress = ReadFileProgress; + func.cardinality = ReadFileCardinality; + func.projection_pushdown = true; + return func; +} + +void ReadBlobFunction::RegisterFunction(BuiltinFunctions &set) { + set.AddFunction(MultiFileReader::CreateFunctionSet(GetFunction())); +} + +void ReadTextFunction::RegisterFunction(BuiltinFunctions &set) { + set.AddFunction(MultiFileReader::CreateFunctionSet(GetFunction())); +} + +} // namespace duckdb diff --git a/src/duckdb/src/function/table/sniff_csv.cpp b/src/duckdb/src/function/table/sniff_csv.cpp index ad1ab7577..450dadf31 100644 --- a/src/duckdb/src/function/table/sniff_csv.cpp +++ b/src/duckdb/src/function/table/sniff_csv.cpp @@ -1,13 +1,12 @@ #include "duckdb/function/built_in_functions.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "duckdb/common/types/data_chunk.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" #include "duckdb/function/table_function.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/function/table/range.hpp" -#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" #include "duckdb/function/table/read_csv.hpp" namespace duckdb { @@ -113,7 +112,7 @@ static void CSVSniffFunction(ClientContext &context, TableFunctionInput &data_p, const CSVSniffFunctionData &data = data_p.bind_data->Cast(); auto &fs = duckdb::FileSystem::GetFileSystem(context); - if (fs.HasGlob(data.path)) { + if (data.path.rfind("http://", 0) != 0 && data.path.rfind("https://", 0) != 0 && fs.HasGlob(data.path)) { throw NotImplementedException("sniff_csv does not operate on globs yet"); } @@ -121,10 +120,8 @@ static void CSVSniffFunction(ClientContext &context, TableFunctionInput &data_p, auto sniffer_options = data.options; sniffer_options.file_path = data.path; - CSVStateMachineCache state_machine_cache; - auto file_handle = BaseCSVReader::OpenCSV(context, sniffer_options); - auto buffer_manager = make_shared(context, std::move(file_handle), sniffer_options); - CSVSniffer sniffer(sniffer_options, buffer_manager, state_machine_cache); + auto buffer_manager = make_shared(context, sniffer_options, sniffer_options.file_path, 0); + CSVSniffer sniffer(sniffer_options, buffer_manager, CSVStateMachineCache::Get(context)); auto sniffer_result = sniffer.SniffCSV(true); string str_opt; string separator = ", "; @@ -141,7 +138,8 @@ static void CSVSniffFunction(ClientContext &context, TableFunctionInput &data_p, str_opt = sniffer_options.dialect_options.state_machine_options.escape.GetValue(); output.SetValue(2, 0, str_opt); // 4. NewLine Delimiter - auto new_line_identifier = NewLineIdentifierToString(sniffer_options.dialect_options.new_line.GetValue()); + auto new_line_identifier = + NewLineIdentifierToString(sniffer_options.dialect_options.state_machine_options.new_line.GetValue()); output.SetValue(3, 0, new_line_identifier); // 5. Skip Rows output.SetValue(4, 0, Value::UINTEGER(sniffer_options.dialect_options.skip_rows.GetValue())); @@ -216,7 +214,7 @@ static void CSVSniffFunction(ClientContext &context, TableFunctionInput &data_p, << separator; } // 11.4. NewLine Delimiter - if (!sniffer_options.dialect_options.new_line.IsSetByUser()) { + if (!sniffer_options.dialect_options.state_machine_options.new_line.IsSetByUser()) { if (new_line_identifier != "mix") { csv_read << "new_line=" << "'" << new_line_identifier << "'" << separator; diff --git a/src/duckdb/src/function/table/system/duckdb_columns.cpp b/src/duckdb/src/function/table/system/duckdb_columns.cpp index fe3e3e637..922d6d32e 100644 --- a/src/duckdb/src/function/table/system/duckdb_columns.cpp +++ b/src/duckdb/src/function/table/system/duckdb_columns.cpp @@ -47,6 +47,9 @@ static unique_ptr DuckDBColumnsBind(ClientContext &context, TableF names.emplace_back("column_index"); return_types.emplace_back(LogicalType::INTEGER); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -102,6 +105,7 @@ class ColumnHelper { virtual const LogicalType &ColumnType(idx_t col) = 0; virtual const Value ColumnDefault(idx_t col) = 0; virtual bool IsNullable(idx_t col) = 0; + virtual const Value ColumnComment(idx_t col) = 0; void WriteColumns(idx_t index, idx_t start_col, idx_t end_col, DataChunk &output); }; @@ -141,6 +145,9 @@ class TableColumnHelper : public ColumnHelper { bool IsNullable(idx_t col) override { return not_null_cols.find(col) == not_null_cols.end(); } + const Value ColumnComment(idx_t col) override { + return entry.GetColumn(LogicalIndex(col)).Comment(); + } private: TableCatalogEntry &entry; @@ -170,6 +177,9 @@ class ViewColumnHelper : public ColumnHelper { bool IsNullable(idx_t col) override { return true; } + const Value ColumnComment(idx_t col) override { + return Value(); + } private: ViewCatalogEntry &entry; @@ -208,6 +218,8 @@ void ColumnHelper::WriteColumns(idx_t start_index, idx_t start_col, idx_t end_co output.SetValue(col++, index, Value(ColumnName(i))); // column_index, INTEGER output.SetValue(col++, index, Value::INTEGER(i + 1)); + // comment, VARCHAR + output.SetValue(col++, index, ColumnComment(i)); // internal, BOOLEAN output.SetValue(col++, index, Value::BOOLEAN(entry.internal)); // column_default, VARCHAR diff --git a/src/duckdb/src/function/table/system/duckdb_databases.cpp b/src/duckdb/src/function/table/system/duckdb_databases.cpp index 981c7f840..79a8e4325 100644 --- a/src/duckdb/src/function/table/system/duckdb_databases.cpp +++ b/src/duckdb/src/function/table/system/duckdb_databases.cpp @@ -23,6 +23,9 @@ static unique_ptr DuckDBDatabasesBind(ClientContext &context, Tabl names.emplace_back("path"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -71,6 +74,8 @@ void DuckDBDatabasesFunction(ClientContext &context, TableFunctionInput &data_p, } } output.SetValue(col++, count, db_path); + // comment, VARCHAR + output.SetValue(col++, count, Value(attached.comment)); // internal, BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(is_internal)); // type, VARCHAR diff --git a/src/duckdb/src/function/table/system/duckdb_functions.cpp b/src/duckdb/src/function/table/system/duckdb_functions.cpp index 2eb6499e7..107da17e2 100644 --- a/src/duckdb/src/function/table/system/duckdb_functions.cpp +++ b/src/duckdb/src/function/table/system/duckdb_functions.cpp @@ -31,6 +31,9 @@ static unique_ptr DuckDBFunctionsBind(ClientContext &context, Tabl names.emplace_back("database_name"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("database_oid"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("schema_name"); return_types.emplace_back(LogicalType::VARCHAR); @@ -43,6 +46,9 @@ static unique_ptr DuckDBFunctionsBind(ClientContext &context, Tabl names.emplace_back("description"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("return_type"); return_types.emplace_back(LogicalType::VARCHAR); @@ -70,6 +76,9 @@ static unique_ptr DuckDBFunctionsBind(ClientContext &context, Tabl names.emplace_back("example"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("stability"); + return_types.emplace_back(LogicalType::VARCHAR); + return nullptr; } @@ -99,6 +108,19 @@ unique_ptr DuckDBFunctionsInit(ClientContext &context, return std::move(result); } +Value FunctionStabilityToValue(FunctionStability stability) { + switch (stability) { + case FunctionStability::VOLATILE: + return Value("VOLATILE"); + case FunctionStability::CONSISTENT: + return Value("CONSISTENT"); + case FunctionStability::CONSISTENT_WITHIN_QUERY: + return Value("CONSISTENT_WITHIN_QUERY"); + default: + throw InternalException("Unsupported FunctionStability"); + } +} + struct ScalarFunctionExtractor { static idx_t FunctionCount(ScalarFunctionCatalogEntry &entry) { return entry.functions.Size(); @@ -138,9 +160,12 @@ struct ScalarFunctionExtractor { return Value(); } - static Value HasSideEffects(ScalarFunctionCatalogEntry &entry, idx_t offset) { - return Value::BOOLEAN(entry.functions.GetFunctionByOffset(offset).side_effects == - FunctionSideEffects::HAS_SIDE_EFFECTS); + static Value IsVolatile(ScalarFunctionCatalogEntry &entry, idx_t offset) { + return Value::BOOLEAN(entry.functions.GetFunctionByOffset(offset).stability == FunctionStability::VOLATILE); + } + + static Value ResultType(ScalarFunctionCatalogEntry &entry, idx_t offset) { + return FunctionStabilityToValue(entry.functions.GetFunctionByOffset(offset).stability); } }; @@ -183,9 +208,12 @@ struct AggregateFunctionExtractor { return Value(); } - static Value HasSideEffects(AggregateFunctionCatalogEntry &entry, idx_t offset) { - return Value::BOOLEAN(entry.functions.GetFunctionByOffset(offset).side_effects == - FunctionSideEffects::HAS_SIDE_EFFECTS); + static Value IsVolatile(AggregateFunctionCatalogEntry &entry, idx_t offset) { + return Value::BOOLEAN(entry.functions.GetFunctionByOffset(offset).stability == FunctionStability::VOLATILE); + } + + static Value ResultType(AggregateFunctionCatalogEntry &entry, idx_t offset) { + return FunctionStabilityToValue(entry.functions.GetFunctionByOffset(offset).stability); } }; @@ -236,7 +264,11 @@ struct MacroExtractor { return func.expression->ToString(); } - static Value HasSideEffects(ScalarMacroCatalogEntry &entry, idx_t offset) { + static Value IsVolatile(ScalarMacroCatalogEntry &entry, idx_t offset) { + return Value(); + } + + static Value ResultType(ScalarMacroCatalogEntry &entry, idx_t offset) { return Value(); } }; @@ -283,14 +315,18 @@ struct TableMacroExtractor { } static Value GetMacroDefinition(TableMacroCatalogEntry &entry, idx_t offset) { - if (entry.function->type == MacroType::SCALAR_MACRO) { - auto &func = entry.function->Cast(); - return func.expression->ToString(); + if (entry.function->type == MacroType::TABLE_MACRO) { + auto &func = entry.function->Cast(); + return func.query_node->ToString(); } return Value(); } - static Value HasSideEffects(TableMacroCatalogEntry &entry, idx_t offset) { + static Value IsVolatile(TableMacroCatalogEntry &entry, idx_t offset) { + return Value(); + } + + static Value ResultType(TableMacroCatalogEntry &entry, idx_t offset) { return Value(); } }; @@ -342,7 +378,11 @@ struct TableFunctionExtractor { return Value(); } - static Value HasSideEffects(TableFunctionCatalogEntry &entry, idx_t offset) { + static Value IsVolatile(TableFunctionCatalogEntry &entry, idx_t offset) { + return Value(); + } + + static Value ResultType(TableFunctionCatalogEntry &entry, idx_t offset) { return Value(); } }; @@ -395,7 +435,11 @@ struct PragmaFunctionExtractor { return Value(); } - static Value HasSideEffects(PragmaFunctionCatalogEntry &entry, idx_t offset) { + static Value IsVolatile(PragmaFunctionCatalogEntry &entry, idx_t offset) { + return Value(); + } + + static Value ResultType(PragmaFunctionCatalogEntry &entry, idx_t offset) { return Value(); } }; @@ -408,6 +452,9 @@ bool ExtractFunctionData(FunctionEntry &entry, idx_t function_idx, DataChunk &ou // database_name, LogicalType::VARCHAR output.SetValue(col++, output_offset, Value(function.schema.catalog.GetName())); + // database_oid, BIGINT + output.SetValue(col++, output_offset, Value::BIGINT(function.schema.catalog.GetOid())); + // schema_name, LogicalType::VARCHAR output.SetValue(col++, output_offset, Value(function.schema.name)); @@ -420,6 +467,9 @@ bool ExtractFunctionData(FunctionEntry &entry, idx_t function_idx, DataChunk &ou // function_description, LogicalType::VARCHAR output.SetValue(col++, output_offset, entry.description.empty() ? Value() : entry.description); + // comment, LogicalType::VARCHAR + output.SetValue(col++, output_offset, entry.comment); + // return_type, LogicalType::VARCHAR output.SetValue(col++, output_offset, OP::GetReturnType(function, function_idx)); @@ -441,7 +491,7 @@ bool ExtractFunctionData(FunctionEntry &entry, idx_t function_idx, DataChunk &ou output.SetValue(col++, output_offset, OP::GetMacroDefinition(function, function_idx)); // has_side_effects, LogicalType::BOOLEAN - output.SetValue(col++, output_offset, OP::HasSideEffects(function, function_idx)); + output.SetValue(col++, output_offset, OP::IsVolatile(function, function_idx)); // internal, LogicalType::BOOLEAN output.SetValue(col++, output_offset, Value::BOOLEAN(function.internal)); @@ -452,6 +502,9 @@ bool ExtractFunctionData(FunctionEntry &entry, idx_t function_idx, DataChunk &ou // example, LogicalType::VARCHAR output.SetValue(col++, output_offset, entry.example.empty() ? Value() : entry.example); + // stability, LogicalType::VARCHAR + output.SetValue(col++, output_offset, OP::ResultType(function, function_idx)); + return function_idx + 1 == OP::FunctionCount(function); } diff --git a/src/duckdb/src/function/table/system/duckdb_indexes.cpp b/src/duckdb/src/function/table/system/duckdb_indexes.cpp index d0ac2a948..7cbb6e904 100644 --- a/src/duckdb/src/function/table/system/duckdb_indexes.cpp +++ b/src/duckdb/src/function/table/system/duckdb_indexes.cpp @@ -42,6 +42,9 @@ static unique_ptr DuckDBIndexesBind(ClientContext &context, TableF names.emplace_back("table_oid"); return_types.emplace_back(LogicalType::BIGINT); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("is_unique"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -104,6 +107,8 @@ void DuckDBIndexesFunction(ClientContext &context, TableFunctionInput &data_p, D output.SetValue(col++, count, Value(table_entry.name)); // table_oid, BIGINT output.SetValue(col++, count, Value::BIGINT(table_entry.oid)); + // comment, VARCHAR + output.SetValue(col++, count, Value(index.comment)); // is_unique, BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(index.IsUnique())); // is_primary, BOOLEAN diff --git a/src/duckdb/src/function/table/system/duckdb_optimizers.cpp b/src/duckdb/src/function/table/system/duckdb_optimizers.cpp index e75d04b0c..ac531678a 100644 --- a/src/duckdb/src/function/table/system/duckdb_optimizers.cpp +++ b/src/duckdb/src/function/table/system/duckdb_optimizers.cpp @@ -1,5 +1,5 @@ #include "duckdb/function/table/system_functions.hpp" -#include "duckdb/common/types/chunk_collection.hpp" + #include "duckdb/main/config.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/common/enum_util.hpp" diff --git a/src/duckdb/src/function/table/system/duckdb_schemas.cpp b/src/duckdb/src/function/table/system/duckdb_schemas.cpp index f0c4e6e4c..65d4a48ca 100644 --- a/src/duckdb/src/function/table/system/duckdb_schemas.cpp +++ b/src/duckdb/src/function/table/system/duckdb_schemas.cpp @@ -30,6 +30,9 @@ static unique_ptr DuckDBSchemasBind(ClientContext &context, TableF names.emplace_back("schema_name"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -70,6 +73,8 @@ void DuckDBSchemasFunction(ClientContext &context, TableFunctionInput &data_p, D output.SetValue(col++, count, Value::BIGINT(entry.catalog.GetOid())); // "schema_name", PhysicalType::VARCHAR output.SetValue(col++, count, Value(entry.name)); + // "comment", PhysicalType::VARCHAR + output.SetValue(col++, count, Value(entry.comment)); // "internal", PhysicalType::BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(entry.internal)); // "sql", PhysicalType::VARCHAR diff --git a/src/duckdb/src/function/table/system/duckdb_secrets.cpp b/src/duckdb/src/function/table/system/duckdb_secrets.cpp new file mode 100644 index 000000000..7cdf0ecb1 --- /dev/null +++ b/src/duckdb/src/function/table/system/duckdb_secrets.cpp @@ -0,0 +1,128 @@ +#include "duckdb/function/table/system_functions.hpp" + +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/map.hpp" +#include "duckdb/common/string_util.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/main/extension_helper.hpp" +#include "duckdb/main/secret/secret_manager.hpp" + +namespace duckdb { + +struct DuckDBSecretsData : public GlobalTableFunctionState { + DuckDBSecretsData() : offset(0) { + } + idx_t offset; +}; + +struct DuckDBSecretsBindData : public FunctionData { +public: + unique_ptr Copy() const override { + return make_uniq(); + }; + + bool Equals(const FunctionData &other_p) const override { + auto &other = other_p.Cast(); + return redact == other.redact; + } + SecretDisplayType redact = SecretDisplayType::REDACTED; +}; + +static unique_ptr DuckDBSecretsBind(ClientContext &context, TableFunctionBindInput &input, + vector &return_types, vector &names) { + auto result = make_uniq(); + + auto entry = input.named_parameters.find("redact"); + if (entry != input.named_parameters.end()) { + if (BooleanValue::Get(entry->second)) { + result->redact = SecretDisplayType::REDACTED; + } else { + result->redact = SecretDisplayType::UNREDACTED; + } + } + + if (!DBConfig::GetConfig(context).options.allow_unredacted_secrets && + result->redact == SecretDisplayType::UNREDACTED) { + throw InvalidInputException("Displaying unredacted secrets is disabled"); + } + + names.emplace_back("name"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("type"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("provider"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("persistent"); + return_types.emplace_back(LogicalType::BOOLEAN); + + names.emplace_back("storage"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("scope"); + return_types.emplace_back(LogicalType::LIST(LogicalType::VARCHAR)); + + names.emplace_back("secret_string"); + return_types.emplace_back(LogicalType::VARCHAR); + + return std::move(result); +} + +unique_ptr DuckDBSecretsInit(ClientContext &context, TableFunctionInitInput &input) { + auto result = make_uniq(); + return std::move(result); +} + +void DuckDBSecretsFunction(ClientContext &context, TableFunctionInput &data_p, DataChunk &output) { + auto &data = data_p.global_state->Cast(); + auto &bind_data = data_p.bind_data->Cast(); + + auto &secret_manager = SecretManager::Get(context); + + auto transaction = CatalogTransaction::GetSystemCatalogTransaction(context); + auto secrets = secret_manager.AllSecrets(transaction); + + if (data.offset >= secrets.size()) { + // finished returning values + return; + } + // start returning values + // either fill up the chunk or return all the remaining columns + idx_t count = 0; + while (data.offset < secrets.size() && count < STANDARD_VECTOR_SIZE) { + auto &secret_entry = secrets[data.offset]; + + vector scope_value; + for (const auto &scope_entry : secret_entry.get().secret->GetScope()) { + scope_value.push_back(scope_entry); + } + + const auto &secret = *secret_entry.get().secret; + + output.SetValue(0, count, secret.GetName()); + output.SetValue(1, count, Value(secret.GetType())); + output.SetValue(2, count, Value(secret.GetProvider())); + output.SetValue(3, count, Value(secret_entry.get().persist_type == SecretPersistType::PERSISTENT)); + output.SetValue(4, count, Value(secret_entry.get().storage_mode)); + output.SetValue(5, count, Value::LIST(LogicalType::VARCHAR, scope_value)); + output.SetValue(6, count, secret.ToString(bind_data.redact)); + + data.offset++; + count++; + } + output.SetCardinality(count); +} + +void DuckDBSecretsFun::RegisterFunction(BuiltinFunctions &set) { + TableFunctionSet functions("duckdb_secrets"); + auto fun = TableFunction({}, DuckDBSecretsFunction, DuckDBSecretsBind, DuckDBSecretsInit); + fun.named_parameters["redact"] = LogicalType::BOOLEAN; + functions.AddFunction(fun); + set.AddFunction(functions); +} + +} // namespace duckdb diff --git a/src/duckdb/src/function/table/system/duckdb_sequences.cpp b/src/duckdb/src/function/table/system/duckdb_sequences.cpp index 985b80944..b4b274a83 100644 --- a/src/duckdb/src/function/table/system/duckdb_sequences.cpp +++ b/src/duckdb/src/function/table/system/duckdb_sequences.cpp @@ -37,6 +37,9 @@ static unique_ptr DuckDBSequencesBind(ClientContext &context, Tabl names.emplace_back("sequence_oid"); return_types.emplace_back(LogicalType::BIGINT); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("temporary"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -87,6 +90,7 @@ void DuckDBSequencesFunction(ClientContext &context, TableFunctionInput &data_p, idx_t count = 0; while (data.offset < data.entries.size() && count < STANDARD_VECTOR_SIZE) { auto &seq = data.entries[data.offset++].get(); + auto seq_data = seq.GetData(); // return values: idx_t col = 0; @@ -102,20 +106,22 @@ void DuckDBSequencesFunction(ClientContext &context, TableFunctionInput &data_p, output.SetValue(col++, count, Value(seq.name)); // sequence_oid, BIGINT output.SetValue(col++, count, Value::BIGINT(seq.oid)); + // comment, VARCHAR + output.SetValue(col++, count, Value(seq.comment)); // temporary, BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(seq.temporary)); // start_value, BIGINT - output.SetValue(col++, count, Value::BIGINT(seq.start_value)); + output.SetValue(col++, count, Value::BIGINT(seq_data.start_value)); // min_value, BIGINT - output.SetValue(col++, count, Value::BIGINT(seq.min_value)); + output.SetValue(col++, count, Value::BIGINT(seq_data.min_value)); // max_value, BIGINT - output.SetValue(col++, count, Value::BIGINT(seq.max_value)); + output.SetValue(col++, count, Value::BIGINT(seq_data.max_value)); // increment_by, BIGINT - output.SetValue(col++, count, Value::BIGINT(seq.increment)); + output.SetValue(col++, count, Value::BIGINT(seq_data.increment)); // cycle, BOOLEAN - output.SetValue(col++, count, Value::BOOLEAN(seq.cycle)); + output.SetValue(col++, count, Value::BOOLEAN(seq_data.cycle)); // last_value, BIGINT - output.SetValue(col++, count, seq.usage_count == 0 ? Value() : Value::BOOLEAN(seq.last_value)); + output.SetValue(col++, count, seq_data.usage_count == 0 ? Value() : Value::BOOLEAN(seq_data.last_value)); // sql, LogicalType::VARCHAR output.SetValue(col++, count, Value(seq.ToSQL())); diff --git a/src/duckdb/src/function/table/system/duckdb_settings.cpp b/src/duckdb/src/function/table/system/duckdb_settings.cpp index c0fa8bbf3..6376ecb46 100644 --- a/src/duckdb/src/function/table/system/duckdb_settings.cpp +++ b/src/duckdb/src/function/table/system/duckdb_settings.cpp @@ -1,5 +1,4 @@ #include "duckdb/function/table/system_functions.hpp" -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/main/config.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/common/enum_util.hpp" diff --git a/src/duckdb/src/function/table/system/duckdb_tables.cpp b/src/duckdb/src/function/table/system/duckdb_tables.cpp index f73ade557..256badef5 100644 --- a/src/duckdb/src/function/table/system/duckdb_tables.cpp +++ b/src/duckdb/src/function/table/system/duckdb_tables.cpp @@ -41,6 +41,9 @@ static unique_ptr DuckDBTablesBind(ClientContext &context, TableFu names.emplace_back("table_oid"); return_types.emplace_back(LogicalType::BIGINT); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -133,6 +136,8 @@ void DuckDBTablesFunction(ClientContext &context, TableFunctionInput &data_p, Da output.SetValue(col++, count, Value(table.name)); // table_oid, LogicalType::BIGINT output.SetValue(col++, count, Value::BIGINT(table.oid)); + // comment, LogicalType::VARCHAR + output.SetValue(col++, count, Value(table.comment)); // internal, LogicalType::BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(table.internal)); // temporary, LogicalType::BOOLEAN diff --git a/src/duckdb/src/function/table/system/duckdb_types.cpp b/src/duckdb/src/function/table/system/duckdb_types.cpp index 7647bcbda..2a7fc1fd1 100644 --- a/src/duckdb/src/function/table/system/duckdb_types.cpp +++ b/src/duckdb/src/function/table/system/duckdb_types.cpp @@ -49,6 +49,9 @@ static unique_ptr DuckDBTypesBind(ClientContext &context, TableFun names.emplace_back("type_category"); return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -129,6 +132,7 @@ void DuckDBTypesFunction(ClientContext &context, TableFunctionInput &data_p, Dat case LogicalTypeId::UINTEGER: case LogicalTypeId::UBIGINT: case LogicalTypeId::HUGEINT: + case LogicalTypeId::UHUGEINT: category = "NUMERIC"; break; case LogicalTypeId::DATE: @@ -159,6 +163,8 @@ void DuckDBTypesFunction(ClientContext &context, TableFunctionInput &data_p, Dat break; } output.SetValue(col++, count, category.empty() ? Value() : Value(category)); + // comment, VARCHAR + output.SetValue(col++, count, Value(type_entry.comment)); // internal, BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(type_entry.internal)); // labels, VARCHAR[] diff --git a/src/duckdb/src/function/table/system/duckdb_views.cpp b/src/duckdb/src/function/table/system/duckdb_views.cpp index 6375db25b..bd17a7ef6 100644 --- a/src/duckdb/src/function/table/system/duckdb_views.cpp +++ b/src/duckdb/src/function/table/system/duckdb_views.cpp @@ -37,6 +37,9 @@ static unique_ptr DuckDBViewsBind(ClientContext &context, TableFun names.emplace_back("view_oid"); return_types.emplace_back(LogicalType::BIGINT); + names.emplace_back("comment"); + return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("internal"); return_types.emplace_back(LogicalType::BOOLEAN); @@ -95,6 +98,8 @@ void DuckDBViewsFunction(ClientContext &context, TableFunctionInput &data_p, Dat output.SetValue(col++, count, Value(view.name)); // view_oid, LogicalType::BIGINT output.SetValue(col++, count, Value::BIGINT(view.oid)); + // comment, LogicalType::VARCHARs + output.SetValue(col++, count, Value(view.comment)); // internal, LogicalType::BOOLEAN output.SetValue(col++, count, Value::BOOLEAN(view.internal)); // temporary, LogicalType::BOOLEAN diff --git a/src/duckdb/src/function/table/system/pragma_table_info.cpp b/src/duckdb/src/function/table/system/pragma_table_info.cpp index a7a7bc4f9..a2d441269 100644 --- a/src/duckdb/src/function/table/system/pragma_table_info.cpp +++ b/src/duckdb/src/function/table/system/pragma_table_info.cpp @@ -17,10 +17,12 @@ namespace duckdb { struct PragmaTableFunctionData : public TableFunctionData { - explicit PragmaTableFunctionData(CatalogEntry &entry_p) : entry(entry_p) { + explicit PragmaTableFunctionData(CatalogEntry &entry_p, bool is_table_info) + : entry(entry_p), is_table_info(is_table_info) { } CatalogEntry &entry; + bool is_table_info; }; struct PragmaTableOperatorData : public GlobalTableFunctionState { @@ -29,63 +31,176 @@ struct PragmaTableOperatorData : public GlobalTableFunctionState { idx_t offset; }; -static unique_ptr PragmaTableInfoBind(ClientContext &context, TableFunctionBindInput &input, - vector &return_types, vector &names) { +struct ColumnConstraintInfo { + bool not_null = false; + bool pk = false; + bool unique = false; +}; + +static Value DefaultValue(const ColumnDefinition &def) { + if (def.Generated()) { + return Value(def.GeneratedExpression().ToString()); + } + if (!def.HasDefaultValue()) { + return Value(); + } + auto &value = def.DefaultValue(); + return Value(value.ToString()); +} + +struct PragmaTableInfoHelper { + static void GetSchema(vector &return_types, vector &names) { + names.emplace_back("cid"); + return_types.emplace_back(LogicalType::INTEGER); - names.emplace_back("cid"); - return_types.emplace_back(LogicalType::INTEGER); + names.emplace_back("name"); + return_types.emplace_back(LogicalType::VARCHAR); - names.emplace_back("name"); - return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("type"); + return_types.emplace_back(LogicalType::VARCHAR); - names.emplace_back("type"); - return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("notnull"); + return_types.emplace_back(LogicalType::BOOLEAN); - names.emplace_back("notnull"); - return_types.emplace_back(LogicalType::BOOLEAN); + names.emplace_back("dflt_value"); + return_types.emplace_back(LogicalType::VARCHAR); - names.emplace_back("dflt_value"); - return_types.emplace_back(LogicalType::VARCHAR); + names.emplace_back("pk"); + return_types.emplace_back(LogicalType::BOOLEAN); + } - names.emplace_back("pk"); - return_types.emplace_back(LogicalType::BOOLEAN); + static void GetTableColumns(const ColumnDefinition &column, ColumnConstraintInfo constraint_info, DataChunk &output, + idx_t index) { + // return values: + // "cid", PhysicalType::INT32 + output.SetValue(0, index, Value::INTEGER((int32_t)column.Oid())); + // "name", PhysicalType::VARCHAR + output.SetValue(1, index, Value(column.Name())); + // "type", PhysicalType::VARCHAR + output.SetValue(2, index, Value(column.Type().ToString())); + // "notnull", PhysicalType::BOOL + output.SetValue(3, index, Value::BOOLEAN(constraint_info.not_null)); + // "dflt_value", PhysicalType::VARCHAR + output.SetValue(4, index, DefaultValue(column)); + // "pk", PhysicalType::BOOL + output.SetValue(5, index, Value::BOOLEAN(constraint_info.pk)); + } + + static void GetViewColumns(idx_t i, const string &name, const LogicalType &type, DataChunk &output, idx_t index) { + // return values: + // "cid", PhysicalType::INT32 + output.SetValue(0, index, Value::INTEGER((int32_t)i)); + // "name", PhysicalType::VARCHAR + output.SetValue(1, index, Value(name)); + // "type", PhysicalType::VARCHAR + output.SetValue(2, index, Value(type.ToString())); + // "notnull", PhysicalType::BOOL + output.SetValue(3, index, Value::BOOLEAN(false)); + // "dflt_value", PhysicalType::VARCHAR + output.SetValue(4, index, Value()); + // "pk", PhysicalType::BOOL + output.SetValue(5, index, Value::BOOLEAN(false)); + } +}; + +struct PragmaShowHelper { + static void GetSchema(vector &return_types, vector &names) { + names.emplace_back("column_name"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("column_type"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("null"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("key"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("default"); + return_types.emplace_back(LogicalType::VARCHAR); + + names.emplace_back("extra"); + return_types.emplace_back(LogicalType::VARCHAR); + } + + static void GetTableColumns(const ColumnDefinition &column, ColumnConstraintInfo constraint_info, DataChunk &output, + idx_t index) { + // "column_name", PhysicalType::VARCHAR + output.SetValue(0, index, Value(column.Name())); + // "column_type", PhysicalType::VARCHAR + output.SetValue(1, index, Value(column.Type().ToString())); + // "null", PhysicalType::VARCHAR + output.SetValue(2, index, Value(constraint_info.not_null ? "NO" : "YES")); + // "key", PhysicalType::VARCHAR + Value key; + if (constraint_info.pk || constraint_info.unique) { + key = Value(constraint_info.pk ? "PRI" : "UNI"); + } + output.SetValue(3, index, key); + // "default", VARCHAR + output.SetValue(4, index, DefaultValue(column)); + // "extra", VARCHAR + output.SetValue(5, index, Value()); + } + + static void GetViewColumns(idx_t i, const string &name, const LogicalType &type, DataChunk &output, idx_t index) { + // "column_name", PhysicalType::VARCHAR + output.SetValue(0, index, Value(name)); + // "column_type", PhysicalType::VARCHAR + output.SetValue(1, index, Value(type.ToString())); + // "null", PhysicalType::VARCHAR + output.SetValue(2, index, Value("YES")); + // "key", PhysicalType::VARCHAR + output.SetValue(3, index, Value()); + // "default", VARCHAR + output.SetValue(4, index, Value()); + // "extra", VARCHAR + output.SetValue(5, index, Value()); + } +}; + +template +static unique_ptr PragmaTableInfoBind(ClientContext &context, TableFunctionBindInput &input, + vector &return_types, vector &names) { + if (IS_PRAGMA_TABLE_INFO) { + PragmaTableInfoHelper::GetSchema(return_types, names); + } else { + PragmaShowHelper::GetSchema(return_types, names); + } auto qname = QualifiedName::Parse(input.inputs[0].GetValue()); // look up the table name in the catalog Binder::BindSchemaOrCatalog(context, qname.catalog, qname.schema); auto &entry = Catalog::GetEntry(context, CatalogType::TABLE_ENTRY, qname.catalog, qname.schema, qname.name); - return make_uniq(entry); + return make_uniq(entry, IS_PRAGMA_TABLE_INFO); } unique_ptr PragmaTableInfoInit(ClientContext &context, TableFunctionInitInput &input) { return make_uniq(); } -static void CheckConstraints(TableCatalogEntry &table, const ColumnDefinition &column, bool &out_not_null, - bool &out_pk) { - out_not_null = false; - out_pk = false; +static ColumnConstraintInfo CheckConstraints(TableCatalogEntry &table, const ColumnDefinition &column) { + ColumnConstraintInfo result; // check all constraints - // FIXME: this is pretty inefficient, it probably doesn't matter for (auto &constraint : table.GetConstraints()) { switch (constraint->type) { case ConstraintType::NOT_NULL: { auto ¬_null = constraint->Cast(); if (not_null.index == column.Logical()) { - out_not_null = true; + result.not_null = true; } break; } case ConstraintType::UNIQUE: { auto &unique = constraint->Cast(); - if (unique.is_primary_key) { - if (unique.index == column.Logical()) { - out_pk = true; - } - if (std::find(unique.columns.begin(), unique.columns.end(), column.GetName()) != unique.columns.end()) { - out_pk = true; - } + bool &constraint_info = unique.is_primary_key ? result.pk : result.unique; + if (unique.index == column.Logical()) { + constraint_info = true; + } + if (std::find(unique.columns.begin(), unique.columns.end(), column.GetName()) != unique.columns.end()) { + constraint_info = true; } break; } @@ -93,20 +208,11 @@ static void CheckConstraints(TableCatalogEntry &table, const ColumnDefinition &c break; } } + return result; } -static Value DefaultValue(const ColumnDefinition &def) { - if (def.Generated()) { - return Value(def.GeneratedExpression().ToString()); - } - if (!def.HasDefaultValue()) { - return Value(); - } - auto &value = def.DefaultValue(); - return Value(value.ToString()); -} - -static void PragmaTableInfoTable(PragmaTableOperatorData &data, TableCatalogEntry &table, DataChunk &output) { +static void PragmaTableInfoTable(PragmaTableOperatorData &data, TableCatalogEntry &table, DataChunk &output, + bool is_table_info) { if (data.offset >= table.GetColumns().LogicalColumnCount()) { // finished returning values return; @@ -117,30 +223,22 @@ static void PragmaTableInfoTable(PragmaTableOperatorData &data, TableCatalogEntr output.SetCardinality(next - data.offset); for (idx_t i = data.offset; i < next; i++) { - bool not_null, pk; auto index = i - data.offset; auto &column = table.GetColumn(LogicalIndex(i)); D_ASSERT(column.Oid() < (idx_t)NumericLimits::Maximum()); - CheckConstraints(table, column, not_null, pk); + auto constraint_info = CheckConstraints(table, column); - // return values: - // "cid", PhysicalType::INT32 - output.SetValue(0, index, Value::INTEGER((int32_t)column.Oid())); - // "name", PhysicalType::VARCHAR - output.SetValue(1, index, Value(column.Name())); - // "type", PhysicalType::VARCHAR - output.SetValue(2, index, Value(column.Type().ToString())); - // "notnull", PhysicalType::BOOL - output.SetValue(3, index, Value::BOOLEAN(not_null)); - // "dflt_value", PhysicalType::VARCHAR - output.SetValue(4, index, DefaultValue(column)); - // "pk", PhysicalType::BOOL - output.SetValue(5, index, Value::BOOLEAN(pk)); + if (is_table_info) { + PragmaTableInfoHelper::GetTableColumns(column, constraint_info, output, index); + } else { + PragmaShowHelper::GetTableColumns(column, constraint_info, output, index); + } } data.offset = next; } -static void PragmaTableInfoView(PragmaTableOperatorData &data, ViewCatalogEntry &view, DataChunk &output) { +static void PragmaTableInfoView(PragmaTableOperatorData &data, ViewCatalogEntry &view, DataChunk &output, + bool is_table_info) { if (data.offset >= view.types.size()) { // finished returning values return; @@ -154,20 +252,12 @@ static void PragmaTableInfoView(PragmaTableOperatorData &data, ViewCatalogEntry auto index = i - data.offset; auto type = view.types[i]; auto &name = view.aliases[i]; - // return values: - // "cid", PhysicalType::INT32 - output.SetValue(0, index, Value::INTEGER((int32_t)i)); - // "name", PhysicalType::VARCHAR - output.SetValue(1, index, Value(name)); - // "type", PhysicalType::VARCHAR - output.SetValue(2, index, Value(type.ToString())); - // "notnull", PhysicalType::BOOL - output.SetValue(3, index, Value::BOOLEAN(false)); - // "dflt_value", PhysicalType::VARCHAR - output.SetValue(4, index, Value()); - // "pk", PhysicalType::BOOL - output.SetValue(5, index, Value::BOOLEAN(false)); + if (is_table_info) { + PragmaTableInfoHelper::GetViewColumns(i, name, type, output, index); + } else { + PragmaShowHelper::GetViewColumns(i, name, type, output, index); + } } data.offset = next; } @@ -177,10 +267,10 @@ static void PragmaTableInfoFunction(ClientContext &context, TableFunctionInput & auto &state = data_p.global_state->Cast(); switch (bind_data.entry.type) { case CatalogType::TABLE_ENTRY: - PragmaTableInfoTable(state, bind_data.entry.Cast(), output); + PragmaTableInfoTable(state, bind_data.entry.Cast(), output, bind_data.is_table_info); break; case CatalogType::VIEW_ENTRY: - PragmaTableInfoView(state, bind_data.entry.Cast(), output); + PragmaTableInfoView(state, bind_data.entry.Cast(), output, bind_data.is_table_info); break; default: throw NotImplementedException("Unimplemented catalog type for pragma_table_info"); @@ -189,7 +279,9 @@ static void PragmaTableInfoFunction(ClientContext &context, TableFunctionInput & void PragmaTableInfo::RegisterFunction(BuiltinFunctions &set) { set.AddFunction(TableFunction("pragma_table_info", {LogicalType::VARCHAR}, PragmaTableInfoFunction, - PragmaTableInfoBind, PragmaTableInfoInit)); + PragmaTableInfoBind, PragmaTableInfoInit)); + set.AddFunction(TableFunction("pragma_show", {LogicalType::VARCHAR}, PragmaTableInfoFunction, + PragmaTableInfoBind, PragmaTableInfoInit)); } } // namespace duckdb diff --git a/src/duckdb/src/function/table/system/test_all_types.cpp b/src/duckdb/src/function/table/system/test_all_types.cpp index a0b856266..da43d20ec 100644 --- a/src/duckdb/src/function/table/system/test_all_types.cpp +++ b/src/duckdb/src/function/table/system/test_all_types.cpp @@ -26,6 +26,7 @@ vector TestAllTypesFun::GetTestTypes(bool use_large_enum) { result.emplace_back(LogicalType::INTEGER, "int"); result.emplace_back(LogicalType::BIGINT, "bigint"); result.emplace_back(LogicalType::HUGEINT, "hugeint"); + result.emplace_back(LogicalType::UHUGEINT, "uhugeint"); result.emplace_back(LogicalType::UTINYINT, "utinyint"); result.emplace_back(LogicalType::USMALLINT, "usmallint"); result.emplace_back(LogicalType::UINTEGER, "uint"); diff --git a/src/duckdb/src/function/table/system_functions.cpp b/src/duckdb/src/function/table/system_functions.cpp index 4df8a6bb7..ee4ca0821 100644 --- a/src/duckdb/src/function/table/system_functions.cpp +++ b/src/duckdb/src/function/table/system_functions.cpp @@ -30,6 +30,7 @@ void BuiltinFunctions::RegisterSQLiteFunctions() { DuckDBDependenciesFun::RegisterFunction(*this); DuckDBExtensionsFun::RegisterFunction(*this); DuckDBOptimizersFun::RegisterFunction(*this); + DuckDBSecretsFun::RegisterFunction(*this); DuckDBSequencesFun::RegisterFunction(*this); DuckDBSettingsFun::RegisterFunction(*this); DuckDBTablesFun::RegisterFunction(*this); diff --git a/src/duckdb/src/function/table/table_scan.cpp b/src/duckdb/src/function/table/table_scan.cpp index 8747f4ded..17bf55f76 100644 --- a/src/duckdb/src/function/table/table_scan.cpp +++ b/src/duckdb/src/function/table/table_scan.cpp @@ -1,21 +1,23 @@ #include "duckdb/function/table/table_scan.hpp" #include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" +#include "duckdb/catalog/dependency_list.hpp" #include "duckdb/common/mutex.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/execution/index/art/art.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/attached_database.hpp" #include "duckdb/main/client_config.hpp" #include "duckdb/optimizer/matcher/expression_matcher.hpp" #include "duckdb/planner/expression/bound_between_expression.hpp" #include "duckdb/planner/expression_iterator.hpp" #include "duckdb/planner/operator/logical_get.hpp" #include "duckdb/storage/data_table.hpp" -#include "duckdb/transaction/local_storage.hpp" -#include "duckdb/transaction/duck_transaction.hpp" -#include "duckdb/main/attached_database.hpp" -#include "duckdb/catalog/dependency_list.hpp" -#include "duckdb/function/function_set.hpp" #include "duckdb/storage/table/scan_state.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/transaction/duck_transaction.hpp" +#include "duckdb/transaction/local_storage.hpp" +#include "duckdb/main/client_data.hpp" namespace duckdb { @@ -77,6 +79,9 @@ static unique_ptr TableScanInitLocal(ExecutionContext & auto &tsgs = gstate->Cast(); result->all_columns.Initialize(context.client, tsgs.scanned_types); } + + result->scan_state.options.force_fetch_row = ClientConfig::GetConfig(context.client).force_fetch_row; + return std::move(result); } @@ -117,6 +122,8 @@ static void TableScanFunc(ClientContext &context, TableFunctionInput &data_p, Da auto &state = data_p.local_state->Cast(); auto &transaction = DuckTransaction::Get(context, bind_data.table.catalog); auto &storage = bind_data.table.GetStorage(); + + state.scan_state.options.force_fetch_row = ClientConfig::GetConfig(context).force_fetch_row; do { if (bind_data.is_create_index) { storage.CreateIndexScan(state.scan_state, output, @@ -222,6 +229,8 @@ static unique_ptr IndexScanInitGlobal(ClientContext &c auto result = make_uniq(row_id_data); auto &local_storage = LocalStorage::Get(context, bind_data.table.catalog); + result->local_storage_state.options.force_fetch_row = ClientConfig::GetConfig(context).force_fetch_row; + result->column_ids.reserve(input.column_ids.size()); for (auto &id : input.column_ids) { result->column_ids.push_back(GetStorageIndex(bind_data.table, id)); @@ -296,116 +305,54 @@ void TableScanPushdownComplexFilter(ClientContext &context, LogicalGet &get, Fun // no indexes or no filters: skip the pushdown return; } + + // Lazily initialize any unknown indexes that might have been loaded by an extension + storage.info->InitializeIndexes(context); + // behold storage.info->indexes.Scan([&](Index &index) { // first rewrite the index expression so the ColumnBindings align with the column bindings of the current table - if (index.unbound_expressions.size() > 1) { + if (index.IsUnknown()) { + // unknown index: skip + return false; + } + + if (index.index_type != ART::TYPE_NAME) { + // only ART indexes are supported for now + return false; + } + + auto &art_index = index.Cast(); + + if (art_index.unbound_expressions.size() > 1) { // NOTE: index scans are not (yet) supported for compound index keys return false; } - auto index_expression = index.unbound_expressions[0]->Copy(); + auto index_expression = art_index.unbound_expressions[0]->Copy(); bool rewrite_possible = true; - RewriteIndexExpression(index, get, *index_expression, rewrite_possible); + RewriteIndexExpression(art_index, get, *index_expression, rewrite_possible); if (!rewrite_possible) { // could not rewrite! return false; } - Value low_value, high_value, equal_value; - ExpressionType low_comparison_type = ExpressionType::INVALID, high_comparison_type = ExpressionType::INVALID; // try to find a matching index for any of the filter expressions + auto &transaction = Transaction::Get(context, bind_data.table.catalog); + for (auto &filter : filters) { - auto &expr = *filter; - - // create a matcher for a comparison with a constant - ComparisonExpressionMatcher matcher; - // match on a comparison type - matcher.expr_type = make_uniq(); - // match on a constant comparison with the indexed expression - matcher.matchers.push_back(make_uniq(*index_expression)); - matcher.matchers.push_back(make_uniq()); - - matcher.policy = SetMatcher::Policy::UNORDERED; - - vector> bindings; - if (matcher.Match(expr, bindings)) { - // range or equality comparison with constant value - // we can use our index here - // bindings[0] = the expression - // bindings[1] = the index expression - // bindings[2] = the constant - auto &comparison = bindings[0].get().Cast(); - auto constant_value = bindings[2].get().Cast().value; - auto comparison_type = comparison.type; - if (comparison.left->type == ExpressionType::VALUE_CONSTANT) { - // the expression is on the right side, we flip them around - comparison_type = FlipComparisonExpression(comparison_type); - } - if (comparison_type == ExpressionType::COMPARE_EQUAL) { - // equality value - // equality overrides any other bounds so we just break here - equal_value = constant_value; - break; - } else if (comparison_type == ExpressionType::COMPARE_GREATERTHANOREQUALTO || - comparison_type == ExpressionType::COMPARE_GREATERTHAN) { - // greater than means this is a lower bound - low_value = constant_value; - low_comparison_type = comparison_type; + auto index_state = art_index.TryInitializeScan(transaction, *index_expression, *filter); + if (index_state != nullptr) { + if (art_index.Scan(transaction, storage, *index_state, STANDARD_VECTOR_SIZE, bind_data.result_ids)) { + // use an index scan! + bind_data.is_index_scan = true; + get.function = TableScanFunction::GetIndexScanFunction(); } else { - // smaller than means this is an upper bound - high_value = constant_value; - high_comparison_type = comparison_type; - } - } else if (expr.type == ExpressionType::COMPARE_BETWEEN) { - // BETWEEN expression - auto &between = expr.Cast(); - if (!between.input->Equals(*index_expression)) { - // expression doesn't match the current index expression - continue; + bind_data.result_ids.clear(); } - if (between.lower->type != ExpressionType::VALUE_CONSTANT || - between.upper->type != ExpressionType::VALUE_CONSTANT) { - // not a constant comparison - continue; - } - low_value = (between.lower->Cast()).value; - low_comparison_type = between.lower_inclusive ? ExpressionType::COMPARE_GREATERTHANOREQUALTO - : ExpressionType::COMPARE_GREATERTHAN; - high_value = (between.upper->Cast()).value; - high_comparison_type = between.upper_inclusive ? ExpressionType::COMPARE_LESSTHANOREQUALTO - : ExpressionType::COMPARE_LESSTHAN; - break; - } - } - if (!equal_value.IsNull() || !low_value.IsNull() || !high_value.IsNull()) { - // we can scan this index using this predicate: try a scan - auto &transaction = Transaction::Get(context, bind_data.table.catalog); - unique_ptr index_state; - if (!equal_value.IsNull()) { - // equality predicate - index_state = - index.InitializeScanSinglePredicate(transaction, equal_value, ExpressionType::COMPARE_EQUAL); - } else if (!low_value.IsNull() && !high_value.IsNull()) { - // two-sided predicate - index_state = index.InitializeScanTwoPredicates(transaction, low_value, low_comparison_type, high_value, - high_comparison_type); - } else if (!low_value.IsNull()) { - // less than predicate - index_state = index.InitializeScanSinglePredicate(transaction, low_value, low_comparison_type); - } else { - D_ASSERT(!high_value.IsNull()); - index_state = index.InitializeScanSinglePredicate(transaction, high_value, high_comparison_type); - } - if (index.Scan(transaction, storage, *index_state, STANDARD_VECTOR_SIZE, bind_data.result_ids)) { - // use an index scan! - bind_data.is_index_scan = true; - get.function = TableScanFunction::GetIndexScanFunction(); - } else { - bind_data.result_ids.clear(); + return true; } - return true; } return false; }); diff --git a/src/duckdb/src/function/table/version/pragma_version.cpp b/src/duckdb/src/function/table/version/pragma_version.cpp index 7b6db819f..d970a8983 100644 --- a/src/duckdb/src/function/table/version/pragma_version.cpp +++ b/src/duckdb/src/function/table/version/pragma_version.cpp @@ -1,12 +1,13 @@ #ifndef DUCKDB_VERSION -#define DUCKDB_VERSION "v0.9.3-dev1565" +#define DUCKDB_VERSION "v0.9.3-dev3731" #endif #ifndef DUCKDB_SOURCE_ID -#define DUCKDB_SOURCE_ID "2414840843" +#define DUCKDB_SOURCE_ID "d4c774b1f1" #endif #include "duckdb/function/table/system_functions.hpp" #include "duckdb/main/database.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/platform.h" #include @@ -64,50 +65,7 @@ const char *DuckDB::LibraryVersion() { } string DuckDB::Platform() { -#if defined(DUCKDB_CUSTOM_PLATFORM) - return DUCKDB_QUOTE_DEFINE(DUCKDB_CUSTOM_PLATFORM); -#endif -#if defined(DUCKDB_WASM_VERSION) - // DuckDB-Wasm requires CUSTOM_PLATFORM to be defined - static_assert(0, "DUCKDB_WASM_VERSION should rely on CUSTOM_PLATFORM being provided"); -#endif - string os = "linux"; -#if INTPTR_MAX == INT64_MAX - string arch = "amd64"; -#elif INTPTR_MAX == INT32_MAX - string arch = "i686"; -#else -#error Unknown pointer size or missing size macros! -#endif - string postfix = ""; - -#ifdef _WIN32 - os = "windows"; -#elif defined(__APPLE__) - os = "osx"; -#elif defined(__FreeBSD__) - os = "freebsd"; -#endif -#if defined(__aarch64__) || defined(__ARM_ARCH_ISA_A64) - arch = "arm64"; -#endif - -#if !defined(_GLIBCXX_USE_CXX11_ABI) || _GLIBCXX_USE_CXX11_ABI == 0 - if (os == "linux") { - postfix = "_gcc4"; - } -#endif -#if defined(__ANDROID__) - postfix += "_android"; // using + because it may also be gcc4 -#endif -#ifdef __MINGW32__ - postfix = "_mingw"; -#endif -// this is used for the windows R builds which use a separate build environment -#ifdef DUCKDB_PLATFORM_RTOOLS - postfix = "_rtools"; -#endif - return os + "_" + arch + postfix; + return DuckDBPlatform(); } struct PragmaPlatformData : public GlobalTableFunctionState { diff --git a/src/duckdb/src/include/duckdb.h b/src/duckdb/src/include/duckdb.h index 3e2492566..eb54f2ed1 100644 --- a/src/duckdb/src/include/duckdb.h +++ b/src/duckdb/src/include/duckdb.h @@ -9,7 +9,7 @@ #pragma once -// duplicate of duckdb/main/winapi.hpp +//! duplicate of duckdb/main/winapi.hpp #ifndef DUCKDB_API #ifdef _WIN32 #if defined(DUCKDB_BUILD_LIBRARY) && !defined(DUCKDB_BUILD_LOADABLE_EXTENSION) @@ -22,7 +22,7 @@ #endif #endif -// duplicate of duckdb/main/winapi.hpp +//! duplicate of duckdb/main/winapi.hpp #ifndef DUCKDB_EXTENSION_API #ifdef _WIN32 #ifdef DUCKDB_BUILD_LOADABLE_EXTENSION @@ -35,10 +35,10 @@ #endif #endif -// API versions -// if no explicit API version is defined, the latest API version is used -// Note that using older API versions (i.e. not using DUCKDB_API_LATEST) is deprecated. -// These will not be supported long-term, and will be removed in future versions. +//! API versions +//! If no explicit API version is defined, the latest API version is used. +//! Note that using older API versions (i.e. not using DUCKDB_API_LATEST) is deprecated. +//! These will not be supported long-term, and will be removed in future versions. #ifndef DUCKDB_API_0_3_1 #define DUCKDB_API_0_3_1 1 #endif @@ -55,17 +55,17 @@ #include #include -#include +#include #ifdef __cplusplus extern "C" { #endif //===--------------------------------------------------------------------===// -// Type Information +// Enums //===--------------------------------------------------------------------===// -typedef uint64_t idx_t; +//! An enum over DuckDB's internal types. typedef enum DUCKDB_TYPE { DUCKDB_TYPE_INVALID = 0, // bool @@ -100,6 +100,8 @@ typedef enum DUCKDB_TYPE { DUCKDB_TYPE_INTERVAL, // duckdb_hugeint DUCKDB_TYPE_HUGEINT, + // duckdb_uhugeint + DUCKDB_TYPE_UHUGEINT, // const char* DUCKDB_TYPE_VARCHAR, // duckdb_blob @@ -126,14 +128,82 @@ typedef enum DUCKDB_TYPE { DUCKDB_TYPE_UNION, // duckdb_bit DUCKDB_TYPE_BIT, + // duckdb_time_tz + DUCKDB_TYPE_TIME_TZ, + // duckdb_timestamp + DUCKDB_TYPE_TIMESTAMP_TZ, } duckdb_type; +//! An enum over the returned state of different functions. +typedef enum { DuckDBSuccess = 0, DuckDBError = 1 } duckdb_state; +//! An enum over the pending state of a pending query result. +typedef enum { + DUCKDB_PENDING_RESULT_READY = 0, + DUCKDB_PENDING_RESULT_NOT_READY = 1, + DUCKDB_PENDING_ERROR = 2, + DUCKDB_PENDING_NO_TASKS_AVAILABLE = 3 +} duckdb_pending_state; +//! An enum over DuckDB's different result types. +typedef enum { + DUCKDB_RESULT_TYPE_INVALID, + DUCKDB_RESULT_TYPE_CHANGED_ROWS, + DUCKDB_RESULT_TYPE_NOTHING, + DUCKDB_RESULT_TYPE_QUERY_RESULT, +} duckdb_result_type; +//! An enum over DuckDB's different statement types. +typedef enum { + DUCKDB_STATEMENT_TYPE_INVALID, + DUCKDB_STATEMENT_TYPE_SELECT, + DUCKDB_STATEMENT_TYPE_INSERT, + DUCKDB_STATEMENT_TYPE_UPDATE, + DUCKDB_STATEMENT_TYPE_EXPLAIN, + DUCKDB_STATEMENT_TYPE_DELETE, + DUCKDB_STATEMENT_TYPE_PREPARE, + DUCKDB_STATEMENT_TYPE_CREATE, + DUCKDB_STATEMENT_TYPE_EXECUTE, + DUCKDB_STATEMENT_TYPE_ALTER, + DUCKDB_STATEMENT_TYPE_TRANSACTION, + DUCKDB_STATEMENT_TYPE_COPY, + DUCKDB_STATEMENT_TYPE_ANALYZE, + DUCKDB_STATEMENT_TYPE_VARIABLE_SET, + DUCKDB_STATEMENT_TYPE_CREATE_FUNC, + DUCKDB_STATEMENT_TYPE_DROP, + DUCKDB_STATEMENT_TYPE_EXPORT, + DUCKDB_STATEMENT_TYPE_PRAGMA, + DUCKDB_STATEMENT_TYPE_VACUUM, + DUCKDB_STATEMENT_TYPE_CALL, + DUCKDB_STATEMENT_TYPE_SET, + DUCKDB_STATEMENT_TYPE_LOAD, + DUCKDB_STATEMENT_TYPE_RELATION, + DUCKDB_STATEMENT_TYPE_EXTENSION, + DUCKDB_STATEMENT_TYPE_LOGICAL_PLAN, + DUCKDB_STATEMENT_TYPE_ATTACH, + DUCKDB_STATEMENT_TYPE_DETACH, + DUCKDB_STATEMENT_TYPE_MULTI, +} duckdb_statement_type; + +//===--------------------------------------------------------------------===// +// General type definitions +//===--------------------------------------------------------------------===// + +//! DuckDB's index type. +typedef uint64_t idx_t; + +//! The callback that will be called to destroy data, e.g., +//! bind data (if any), init data (if any), extra data for replacement scans (if any) +typedef void (*duckdb_delete_callback_t)(void *data); + +//! Used for threading, contains a task state. Must be destroyed with `duckdb_destroy_state`. +typedef void *duckdb_task_state; + +//===--------------------------------------------------------------------===// +// Types (no explicit freeing) +//===--------------------------------------------------------------------===// //! Days are stored as days since 1970-01-01 //! Use the duckdb_from_date/duckdb_to_date function to extract individual information typedef struct { int32_t days; } duckdb_date; - typedef struct { int32_t year; int8_t month; @@ -145,7 +215,6 @@ typedef struct { typedef struct { int64_t micros; } duckdb_time; - typedef struct { int8_t hour; int8_t min; @@ -153,51 +222,60 @@ typedef struct { int32_t micros; } duckdb_time_struct; +//! TIME_TZ is stored as 40 bits for int64_t micros, and 24 bits for int32_t offset +typedef struct { + uint64_t bits; +} duckdb_time_tz; +typedef struct { + duckdb_time time; + int32_t offset; +} duckdb_time_tz_struct; + //! Timestamps are stored as microseconds since 1970-01-01 //! Use the duckdb_from_timestamp/duckdb_to_timestamp function to extract individual information typedef struct { int64_t micros; } duckdb_timestamp; - typedef struct { duckdb_date_struct date; duckdb_time_struct time; } duckdb_timestamp_struct; - typedef struct { int32_t months; int32_t days; int64_t micros; } duckdb_interval; -//! Hugeints are composed in a (lower, upper) component +//! Hugeints are composed of a (lower, upper) component //! The value of the hugeint is upper * 2^64 + lower //! For easy usage, the functions duckdb_hugeint_to_double/duckdb_double_to_hugeint are recommended typedef struct { uint64_t lower; int64_t upper; } duckdb_hugeint; +typedef struct { + uint64_t lower; + uint64_t upper; +} duckdb_uhugeint; +//! Decimals are composed of a width and a scale, and are stored in a hugeint typedef struct { uint8_t width; uint8_t scale; - duckdb_hugeint value; } duckdb_decimal; -typedef struct { - char *data; - idx_t size; -} duckdb_string; - +//! A type holding information about the query execution progress typedef struct { double percentage; uint64_t rows_processed; uint64_t total_rows_to_process; } duckdb_query_progress_type; -/* - The internal data representation of a VARCHAR/BLOB column -*/ + +//! The internal representation of a VARCHAR (string_t). If the VARCHAR does not +//! exceed 12 characters, then we inline it. Otherwise, we inline a prefix for faster +//! string comparisons and store a pointer to the remaining characters. This is a non- +//! owning structure, i.e., it does not have to be freed. typedef struct { union { struct { @@ -212,16 +290,18 @@ typedef struct { } value; } duckdb_string_t; -typedef struct { - void *data; - idx_t size; -} duckdb_blob; - +//! The internal representation of a list metadata entry contains the list's offset in +//! the child vector, and its length. The parent vector holds these metadata entries, +//! whereas the child vector holds the data typedef struct { uint64_t offset; uint64_t length; } duckdb_list_entry; +//! A column consists of a pointer to its internal data. Don't operate on this type directly. +//! Instead, use functions such as duckdb_column_data, duckdb_nullmask_data, +//! duckdb_column_type, and duckdb_column_name, which take the result and the column index +//! as their parameters typedef struct { #if DUCKDB_API_VERSION < DUCKDB_API_0_3_2 void *data; @@ -241,6 +321,32 @@ typedef struct { void *internal_data; } duckdb_column; +//! A vector to a specified column in a data chunk. Lives as long as the +//! data chunk lives, i.e., must not be destroyed. +typedef struct _duckdb_vector { + void *__vctr; +} * duckdb_vector; + +//===--------------------------------------------------------------------===// +// Types (explicit freeing/destroying) +//===--------------------------------------------------------------------===// + +//! Strings are composed of a char pointer and a size. You must free string.data +//! with `duckdb_free`. +typedef struct { + char *data; + idx_t size; +} duckdb_string; + +//! BLOBs are composed of a byte pointer and a size. You must free blob.data +//! with `duckdb_free`. +typedef struct { + void *data; + idx_t size; +} duckdb_blob; + +//! A query result consists of a pointer to its internal data. +//! Must be freed with 'duckdb_destroy_result'. typedef struct { #if DUCKDB_API_VERSION < DUCKDB_API_0_3_2 idx_t column_count; @@ -255,7 +361,7 @@ typedef struct { idx_t __deprecated_row_count; // deprecated, use duckdb_rows_changed idx_t __deprecated_rows_changed; - // deprecated, use duckdb_column_ family of functions + // deprecated, use duckdb_column_*-family of functions duckdb_column *__deprecated_columns; // deprecated, use duckdb_result_error char *__deprecated_error_message; @@ -263,98 +369,125 @@ typedef struct { void *internal_data; } duckdb_result; +//! A database object. Should be closed with `duckdb_close`. typedef struct _duckdb_database { void *__db; } * duckdb_database; + +//! A connection to a duckdb database. Must be closed with `duckdb_disconnect`. typedef struct _duckdb_connection { void *__conn; } * duckdb_connection; + +//! A prepared statement is a parameterized query that allows you to bind parameters to it. +//! Must be destroyed with `duckdb_destroy_prepare`. typedef struct _duckdb_prepared_statement { void *__prep; } * duckdb_prepared_statement; + +//! Extracted statements. Must be destroyed with `duckdb_destroy_extracted`. typedef struct _duckdb_extracted_statements { void *__extrac; } * duckdb_extracted_statements; + +//! The pending result represents an intermediate structure for a query that is not yet fully executed. +//! Must be destroyed with `duckdb_destroy_pending`. typedef struct _duckdb_pending_result { void *__pend; } * duckdb_pending_result; + +//! The appender enables fast data loading into DuckDB. +//! Must be destroyed with `duckdb_appender_destroy`. typedef struct _duckdb_appender { void *__appn; } * duckdb_appender; -typedef struct _duckdb_arrow { - void *__arrw; -} * duckdb_arrow; -typedef struct _duckdb_arrow_stream { - void *__arrwstr; -} * duckdb_arrow_stream; + +//! Can be used to provide start-up options for the DuckDB instance. +//! Must be destroyed with `duckdb_destroy_config`. typedef struct _duckdb_config { void *__cnfg; } * duckdb_config; -typedef struct _duckdb_arrow_schema { - void *__arrs; -} * duckdb_arrow_schema; -typedef struct _duckdb_arrow_array { - void *__arra; -} * duckdb_arrow_array; + +//! Holds an internal logical type. +//! Must be destroyed with `duckdb_destroy_logical_type`. typedef struct _duckdb_logical_type { void *__lglt; } * duckdb_logical_type; + +//! Contains a data chunk from a duckdb_result. +//! Must be destroyed with `duckdb_destroy_data_chunk`. typedef struct _duckdb_data_chunk { void *__dtck; } * duckdb_data_chunk; -typedef struct _duckdb_vector { - void *__vctr; -} * duckdb_vector; + +//! Holds a DuckDB value, which wraps a type. +//! Must be destroyed with `duckdb_destroy_value`. typedef struct _duckdb_value { void *__val; } * duckdb_value; -typedef enum { DuckDBSuccess = 0, DuckDBError = 1 } duckdb_state; -typedef enum { - DUCKDB_PENDING_RESULT_READY = 0, - DUCKDB_PENDING_RESULT_NOT_READY = 1, - DUCKDB_PENDING_ERROR = 2, - DUCKDB_PENDING_NO_TASKS_AVAILABLE = 3 -} duckdb_pending_state; +//===--------------------------------------------------------------------===// +// Table function types +//===--------------------------------------------------------------------===// -typedef enum { - DUCKDB_RESULT_TYPE_INVALID, - DUCKDB_RESULT_TYPE_CHANGED_ROWS, - DUCKDB_RESULT_TYPE_NOTHING, - DUCKDB_RESULT_TYPE_QUERY_RESULT, -} duckdb_result_type; +//! A table function. Must be destroyed with `duckdb_destroy_table_function`. +typedef void *duckdb_table_function; -typedef enum { - DUCKDB_STATEMENT_TYPE_INVALID, - DUCKDB_STATEMENT_TYPE_SELECT, - DUCKDB_STATEMENT_TYPE_INSERT, - DUCKDB_STATEMENT_TYPE_UPDATE, - DUCKDB_STATEMENT_TYPE_EXPLAIN, - DUCKDB_STATEMENT_TYPE_DELETE, - DUCKDB_STATEMENT_TYPE_PREPARE, - DUCKDB_STATEMENT_TYPE_CREATE, - DUCKDB_STATEMENT_TYPE_EXECUTE, - DUCKDB_STATEMENT_TYPE_ALTER, - DUCKDB_STATEMENT_TYPE_TRANSACTION, - DUCKDB_STATEMENT_TYPE_COPY, - DUCKDB_STATEMENT_TYPE_ANALYZE, - DUCKDB_STATEMENT_TYPE_VARIABLE_SET, - DUCKDB_STATEMENT_TYPE_CREATE_FUNC, - DUCKDB_STATEMENT_TYPE_DROP, - DUCKDB_STATEMENT_TYPE_EXPORT, - DUCKDB_STATEMENT_TYPE_PRAGMA, - DUCKDB_STATEMENT_TYPE_SHOW, - DUCKDB_STATEMENT_TYPE_VACUUM, - DUCKDB_STATEMENT_TYPE_CALL, - DUCKDB_STATEMENT_TYPE_SET, - DUCKDB_STATEMENT_TYPE_LOAD, - DUCKDB_STATEMENT_TYPE_RELATION, - DUCKDB_STATEMENT_TYPE_EXTENSION, - DUCKDB_STATEMENT_TYPE_LOGICAL_PLAN, - DUCKDB_STATEMENT_TYPE_ATTACH, - DUCKDB_STATEMENT_TYPE_DETACH, - DUCKDB_STATEMENT_TYPE_MULTI, -} duckdb_statement_type; +//! The bind info of the function. When setting this info, it is necessary to pass a destroy-callback function. +typedef void *duckdb_bind_info; + +//! Additional function init info. When setting this info, it is necessary to pass a destroy-callback function. +typedef void *duckdb_init_info; + +//! Additional function info. When setting this info, it is necessary to pass a destroy-callback function. +typedef void *duckdb_function_info; + +//! The bind function of the table function. +typedef void (*duckdb_table_function_bind_t)(duckdb_bind_info info); + +//! The (possibly thread-local) init function of the table function. +typedef void (*duckdb_table_function_init_t)(duckdb_init_info info); + +//! The main function of the table function. +typedef void (*duckdb_table_function_t)(duckdb_function_info info, duckdb_data_chunk output); + +//===--------------------------------------------------------------------===// +// Replacement scan types +//===--------------------------------------------------------------------===// + +//! Additional replacement scan info. When setting this info, it is necessary to pass a destroy-callback function. +typedef void *duckdb_replacement_scan_info; + +//! A replacement scan function that can be added to a database. +typedef void (*duckdb_replacement_callback_t)(duckdb_replacement_scan_info info, const char *table_name, void *data); + +//===--------------------------------------------------------------------===// +// Arrow-related types +//===--------------------------------------------------------------------===// + +//! Holds an arrow query result. Must be destroyed with `duckdb_destroy_arrow`. +typedef struct _duckdb_arrow { + void *__arrw; +} * duckdb_arrow; + +//! Holds an arrow array stream. Must be destroyed with `duckdb_destroy_arrow_stream`. +typedef struct _duckdb_arrow_stream { + void *__arrwstr; +} * duckdb_arrow_stream; + +//! Holds an arrow schema. Remember to release the respective ArrowSchema object. +typedef struct _duckdb_arrow_schema { + void *__arrs; +} * duckdb_arrow_schema; + +//! Holds an arrow array. Remember to release the respective ArrowArray object. +typedef struct _duckdb_arrow_array { + void *__arra; +} * duckdb_arrow_array; + +//===--------------------------------------------------------------------===// +// Functions +//===--------------------------------------------------------------------===// //===--------------------------------------------------------------------===// // Open/Connect @@ -363,7 +496,7 @@ typedef enum { /*! Creates a new database or opens an existing database file stored at the given path. If no path is given a new in-memory database is created instead. -The instantiated database should be closed with 'duckdb_close' +The instantiated database should be closed with 'duckdb_close'. * path: Path to the database file on disk, or `nullptr` or `:memory:` to open an in-memory database. * out_database: The result database object. @@ -373,6 +506,7 @@ DUCKDB_API duckdb_state duckdb_open(const char *path, duckdb_database *out_datab /*! Extended version of duckdb_open. Creates a new database or opens an existing database file stored at the given path. +The instantiated database should be closed with 'duckdb_close'. * path: Path to the database file on disk, or `nullptr` or `:memory:` to open an in-memory database. * out_database: The result database object. @@ -386,9 +520,9 @@ DUCKDB_API duckdb_state duckdb_open_ext(const char *path, duckdb_database *out_d /*! Closes the specified database and de-allocates all memory allocated for that database. -This should be called after you are done with any database allocated through `duckdb_open`. +This should be called after you are done with any database allocated through `duckdb_open` or `duckdb_open_ext`. Note that failing to call `duckdb_close` (in case of e.g. a program crash) will not cause data corruption. -Still it is recommended to always correctly close a database object after you are done with it. +Still, it is recommended to always correctly close a database object after you are done with it. * database: The database object to shut down. */ @@ -397,7 +531,7 @@ DUCKDB_API void duckdb_close(duckdb_database *database); /*! Opens a connection to a database. Connections are required to query the database, and store transactional state associated with the connection. -The instantiated connection should be closed using 'duckdb_disconnect' +The instantiated connection should be closed using 'duckdb_disconnect'. * database: The database file to connect to. * out_connection: The result connection object. @@ -408,7 +542,7 @@ DUCKDB_API duckdb_state duckdb_connect(duckdb_database database, duckdb_connecti /*! Interrupt running query -* connection: The connection to interruot +* connection: The connection to interrupt */ DUCKDB_API void duckdb_interrupt(duckdb_connection connection); @@ -437,9 +571,11 @@ DUCKDB_API const char *duckdb_library_version(); //===--------------------------------------------------------------------===// // Configuration //===--------------------------------------------------------------------===// + /*! Initializes an empty configuration object that can be used to provide start-up options for the DuckDB instance through `duckdb_open_ext`. +The duckdb_config must be destroyed using 'duckdb_destroy_config' This will always succeed unless there is a malloc failure. @@ -486,7 +622,7 @@ This can fail if either the name is invalid, or if the value provided for the op DUCKDB_API duckdb_state duckdb_set_config(duckdb_config config, const char *name, const char *option); /*! -Destroys the specified configuration option and de-allocates all memory allocated for the object. +Destroys the specified configuration object and de-allocates all memory allocated for the object. * config: The configuration object to destroy. */ @@ -495,6 +631,7 @@ DUCKDB_API void duckdb_destroy_config(duckdb_config *config); //===--------------------------------------------------------------------===// // Query Execution //===--------------------------------------------------------------------===// + /*! Executes a SQL query within a connection and stores the full (materialized) result in the out_result pointer. If the query fails to execute, DuckDBError is returned and the error message can be retrieved by calling @@ -518,7 +655,7 @@ Closes the result and de-allocates all memory allocated for that connection. DUCKDB_API void duckdb_destroy_result(duckdb_result *result); /*! -Returns the column name of the specified column. The result should not need be freed; the column names will +Returns the column name of the specified column. The result should not need to be freed; the column names will automatically be destroyed when the result is destroyed. Returns `NULL` if the column is out of range. @@ -570,7 +707,7 @@ Returns the number of columns present in a the result object. DUCKDB_API idx_t duckdb_column_count(duckdb_result *result); /*! -Returns the number of rows present in a the result object. +Returns the number of rows present in the result object. * result: The result object. * returns: The number of rows present in the result object. @@ -687,7 +824,10 @@ Returns the return_type of the given result, or DUCKDB_RETURN_TYPE_INVALID on er */ DUCKDB_API duckdb_result_type duckdb_result_return_type(duckdb_result result); +//===--------------------------------------------------------------------===// // Safe fetch functions +//===--------------------------------------------------------------------===// + // These functions will perform conversions if necessary. // On failure (e.g. if conversion cannot be performed or if the value is NULL) a default value is returned. // Note that these functions are slow since they perform bounds checking and conversion @@ -723,6 +863,11 @@ DUCKDB_API int64_t duckdb_value_int64(duckdb_result *result, idx_t col, idx_t ro */ DUCKDB_API duckdb_hugeint duckdb_value_hugeint(duckdb_result *result, idx_t col, idx_t row); +/*! + * returns: The duckdb_uhugeint value at the specified location, or 0 if the value cannot be converted. + */ +DUCKDB_API duckdb_uhugeint duckdb_value_uhugeint(duckdb_result *result, idx_t col, idx_t row); + /*! * returns: The duckdb_decimal value at the specified location, or 0 if the value cannot be converted. */ @@ -785,10 +930,10 @@ converted. The result must be freed with `duckdb_free`. */ DUCKDB_API char *duckdb_value_varchar(duckdb_result *result, idx_t col, idx_t row); -/*!s -* returns: The string value at the specified location. -The result must be freed with `duckdb_free`. -*/ +/*! + * returns: The string value at the specified location. + * The resulting field "string.data" must be freed with `duckdb_free.` + */ DUCKDB_API duckdb_string duckdb_value_string(duckdb_result *result, idx_t col, idx_t row); /*! @@ -813,7 +958,7 @@ DUCKDB_API duckdb_string duckdb_value_string_internal(duckdb_result *result, idx /*! * returns: The duckdb_blob value at the specified location. Returns a blob with blob.data set to nullptr if the -value cannot be converted. The resulting "blob.data" must be freed with `duckdb_free.` +value cannot be converted. The resulting field "blob.data" must be freed with `duckdb_free.` */ DUCKDB_API duckdb_blob duckdb_value_blob(duckdb_result *result, idx_t col, idx_t row); @@ -825,6 +970,7 @@ DUCKDB_API bool duckdb_value_is_null(duckdb_result *result, idx_t col, idx_t row //===--------------------------------------------------------------------===// // Helpers //===--------------------------------------------------------------------===// + /*! Allocate `size` bytes of memory using the duckdb internal malloc function. Any memory allocated in this manner should be freed using `duckdb_free`. @@ -835,7 +981,8 @@ should be freed using `duckdb_free`. DUCKDB_API void *duckdb_malloc(size_t size); /*! -Free a value returned from `duckdb_malloc`, `duckdb_value_varchar` or `duckdb_value_blob`. +Free a value returned from `duckdb_malloc`, `duckdb_value_varchar`, `duckdb_value_blob`, or +`duckdb_value_string`. * ptr: The memory region to de-allocate. */ @@ -859,6 +1006,7 @@ DUCKDB_API bool duckdb_string_is_inlined(duckdb_string_t string); //===--------------------------------------------------------------------===// // Date/Time/Timestamp Helpers //===--------------------------------------------------------------------===// + /*! Decompose a `duckdb_date` object into year, month and date (stored as `duckdb_date_struct`). @@ -875,6 +1023,14 @@ Re-compose a `duckdb_date` from year, month and date (`duckdb_date_struct`). */ DUCKDB_API duckdb_date duckdb_to_date(duckdb_date_struct date); +/*! +Test a `duckdb_date` to see if it is a finite value. + +* date: The date object, as obtained from a `DUCKDB_TYPE_DATE` column. +* returns: True if the date is finite, false if it is ±infinity. +*/ +DUCKDB_API bool duckdb_is_finite_date(duckdb_date date); + /*! Decompose a `duckdb_time` object into hour, minute, second and microsecond (stored as `duckdb_time_struct`). @@ -883,6 +1039,26 @@ Decompose a `duckdb_time` object into hour, minute, second and microsecond (stor */ DUCKDB_API duckdb_time_struct duckdb_from_time(duckdb_time time); +/*! +Create a `duckdb_time_tz` object from micros and a timezone offset. + +* micros: The microsecond component of the time. +* offset: The timezone offset component of the time. +* returns: The `duckdb_time_tz` element. +*/ +DUCKDB_API duckdb_time_tz duckdb_create_time_tz(int64_t micros, int32_t offset); + +/*! +Decompose a TIME_TZ objects into micros and a timezone offset. + +Use `duckdb_from_time` to further decompose the micros into hour, minute, second and microsecond. + +* micros: The time object, as obtained from a `DUCKDB_TYPE_TIME_TZ` column. +* out_micros: The microsecond component of the time. +* out_offset: The timezone offset component of the time. +*/ +DUCKDB_API duckdb_time_tz_struct duckdb_from_time_tz(duckdb_time_tz micros); + /*! Re-compose a `duckdb_time` from hour, minute, second and microsecond (`duckdb_time_struct`). @@ -907,9 +1083,18 @@ Re-compose a `duckdb_timestamp` from a duckdb_timestamp_struct. */ DUCKDB_API duckdb_timestamp duckdb_to_timestamp(duckdb_timestamp_struct ts); +/*! +Test a `duckdb_timestamp` to see if it is a finite value. + +* ts: The timestamp object, as obtained from a `DUCKDB_TYPE_TIMESTAMP` column. +* returns: True if the timestamp is finite, false if it is ±infinity. +*/ +DUCKDB_API bool duckdb_is_finite_timestamp(duckdb_timestamp ts); + //===--------------------------------------------------------------------===// // Hugeint Helpers //===--------------------------------------------------------------------===// + /*! Converts a duckdb_hugeint object (as obtained from a `DUCKDB_TYPE_HUGEINT` column) into a double. @@ -928,6 +1113,32 @@ If the conversion fails because the double value is too big the result will be 0 */ DUCKDB_API duckdb_hugeint duckdb_double_to_hugeint(double val); +//===--------------------------------------------------------------------===// +// Unsigned Hugeint Helpers +//===--------------------------------------------------------------------===// + +/*! +Converts a duckdb_uhugeint object (as obtained from a `DUCKDB_TYPE_UHUGEINT` column) into a double. + +* val: The uhugeint value. +* returns: The converted `double` element. +*/ +DUCKDB_API double duckdb_uhugeint_to_double(duckdb_uhugeint val); + +/*! +Converts a double value to a duckdb_uhugeint object. + +If the conversion fails because the double value is too big the result will be 0. + +* val: The double value. +* returns: The converted `duckdb_uhugeint` element. +*/ +DUCKDB_API duckdb_uhugeint duckdb_double_to_uhugeint(double val); + +//===--------------------------------------------------------------------===// +// Decimal Helpers +//===--------------------------------------------------------------------===// + /*! Converts a double value to a duckdb_decimal object. @@ -938,9 +1149,6 @@ If the conversion fails because the double value is too big, or the width/scale */ DUCKDB_API duckdb_decimal duckdb_double_to_decimal(double val, uint8_t width, uint8_t scale); -//===--------------------------------------------------------------------===// -// Decimal Helpers -//===--------------------------------------------------------------------===// /*! Converts a duckdb_decimal object (as obtained from a `DUCKDB_TYPE_DECIMAL` column) into a double. @@ -952,6 +1160,7 @@ DUCKDB_API double duckdb_decimal_to_double(duckdb_decimal val); //===--------------------------------------------------------------------===// // Prepared Statements //===--------------------------------------------------------------------===// + // A prepared statement is a parameterized query that allows you to bind parameters to it. // * This is useful to easily supply parameters to functions and avoid SQL injection attacks. // * This is useful to speed up queries that you will execute several times with different parameters. @@ -1039,6 +1248,10 @@ Returns the statement type of the statement to be executed */ DUCKDB_API duckdb_statement_type duckdb_prepared_statement_type(duckdb_prepared_statement statement); +//===--------------------------------------------------------------------===// +// Bind Values to Prepared Statements +//===--------------------------------------------------------------------===// + /*! Binds a value to the prepared statement at the specified index. */ @@ -1082,6 +1295,11 @@ Binds a duckdb_hugeint value to the prepared statement at the specified index. DUCKDB_API duckdb_state duckdb_bind_hugeint(duckdb_prepared_statement prepared_statement, idx_t param_idx, duckdb_hugeint val); /*! +Binds an duckdb_uhugeint value to the prepared statement at the specified index. +*/ +DUCKDB_API duckdb_state duckdb_bind_uhugeint(duckdb_prepared_statement prepared_statement, idx_t param_idx, + duckdb_uhugeint val); +/*! Binds a duckdb_decimal value to the prepared statement at the specified index. */ DUCKDB_API duckdb_state duckdb_bind_decimal(duckdb_prepared_statement prepared_statement, idx_t param_idx, @@ -1164,12 +1382,18 @@ Binds a NULL value to the prepared statement at the specified index. */ DUCKDB_API duckdb_state duckdb_bind_null(duckdb_prepared_statement prepared_statement, idx_t param_idx); +//===--------------------------------------------------------------------===// +// Execute Prepared Statements +//===--------------------------------------------------------------------===// + /*! Executes the prepared statement with the given bound parameters, and returns a materialized query result. This method can be called multiple times for each prepared statement, and the parameters can be modified between calls to this function. +Note that the result must be freed with `duckdb_destroy_result`. + * prepared_statement: The prepared statement to execute. * out_result: The query result. * returns: `DuckDBSuccess` on success or `DuckDBError` on failure. @@ -1184,6 +1408,8 @@ To determine if the resulting query was in fact streamed, use `duckdb_result_is_ This method can be called multiple times for each prepared statement, and the parameters can be modified between calls to this function. +Note that the result must be freed with `duckdb_destroy_result`. + * prepared_statement: The prepared statement to execute. * out_result: The query result. * returns: `DuckDBSuccess` on success or `DuckDBError` on failure. @@ -1191,51 +1417,19 @@ between calls to this function. DUCKDB_API duckdb_state duckdb_execute_prepared_streaming(duckdb_prepared_statement prepared_statement, duckdb_result *out_result); -/*! -Executes the prepared statement with the given bound parameters, and returns an arrow query result. - -* prepared_statement: The prepared statement to execute. -* out_result: The query result. -* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. -*/ -DUCKDB_API duckdb_state duckdb_execute_prepared_arrow(duckdb_prepared_statement prepared_statement, - duckdb_arrow *out_result); - -/*! -Scans the Arrow stream and creates a view with the given name. - -* connection: The connection on which to execute the scan. -* table_name: Name of the temporary view to create. -* arrow: Arrow stream wrapper. -* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. -*/ -DUCKDB_API duckdb_state duckdb_arrow_scan(duckdb_connection connection, const char *table_name, - duckdb_arrow_stream arrow); - -/*! -Scans the Arrow array and creates a view with the given name. - -* connection: The connection on which to execute the scan. -* table_name: Name of the temporary view to create. -* arrow_schema: Arrow schema wrapper. -* arrow_array: Arrow array wrapper. -* out_stream: Output array stream that wraps around the passed schema, for releasing/deleting once done. -* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. -*/ -DUCKDB_API duckdb_state duckdb_arrow_array_scan(duckdb_connection connection, const char *table_name, - duckdb_arrow_schema arrow_schema, duckdb_arrow_array arrow_array, - duckdb_arrow_stream *out_stream); - //===--------------------------------------------------------------------===// // Extract Statements //===--------------------------------------------------------------------===// + // A query string can be extracted into multiple SQL statements. Each statement can be prepared and executed separately. /*! Extract all statements from a query. Note that after calling `duckdb_extract_statements`, the extracted statements should always be destroyed using `duckdb_destroy_extracted`, even if no statements were extracted. + If the extract fails, `duckdb_extract_statements_error` can be called to obtain the reason why the extract failed. + * connection: The connection object * query: The SQL query to extract * out_extracted_statements: The resulting extracted statements object @@ -1248,7 +1442,9 @@ DUCKDB_API idx_t duckdb_extract_statements(duckdb_connection connection, const c Prepare an extracted statement. Note that after calling `duckdb_prepare_extracted_statement`, the prepared statement should always be destroyed using `duckdb_destroy_prepare`, even if the prepare fails. + If the prepare fails, `duckdb_prepare_error` can be called to obtain the reason why the prepare failed. + * connection: The connection object * extracted_statements: The extracted statements object * index: The index of the extracted statement to prepare @@ -1262,6 +1458,7 @@ DUCKDB_API duckdb_state duckdb_prepare_extracted_statement(duckdb_connection con /*! Returns the error message contained within the extracted statements. The result of this function must not be freed. It will be cleaned up when `duckdb_destroy_extracted` is called. + * result: The extracted statements to fetch the error from. * returns: The error of the extracted statements. */ @@ -1276,6 +1473,7 @@ DUCKDB_API void duckdb_destroy_extracted(duckdb_extracted_statements *extracted_ //===--------------------------------------------------------------------===// // Pending Result Interface //===--------------------------------------------------------------------===// + /*! Executes the prepared statement with the given bound parameters, and returns a pending result. The pending result represents an intermediate structure for a query that is not yet fully executed. @@ -1332,7 +1530,7 @@ If this returns DUCKDB_PENDING_ERROR, an error occurred during execution. The error message can be obtained by calling duckdb_pending_error on the pending_result. -* pending_result: The pending result to execute a task within.. +* pending_result: The pending result to execute a task within. * returns: The state of the pending result after the execution. */ DUCKDB_API duckdb_pending_state duckdb_pending_execute_task(duckdb_pending_result pending_result); @@ -1343,6 +1541,8 @@ Fully execute a pending query result, returning the final query result. If duckdb_pending_execute_task has been called until DUCKDB_PENDING_RESULT_READY was returned, this will return fast. Otherwise, all remaining tasks must be executed first. +Note that the result must be freed with `duckdb_destroy_result`. + * pending_result: The pending result to execute. * out_result: The result object. * returns: `DuckDBSuccess` on success or `DuckDBError` on failure. @@ -1361,6 +1561,7 @@ DUCKDB_API bool duckdb_pending_execution_is_finished(duckdb_pending_state pendin //===--------------------------------------------------------------------===// // Value Interface //===--------------------------------------------------------------------===// + /*! Destroys the value and de-allocates all memory allocated for that type. @@ -1445,8 +1646,8 @@ This should not be used with `DUCKDB_TYPE_DECIMAL`. DUCKDB_API duckdb_logical_type duckdb_create_logical_type(duckdb_type type); /*! -Returns the alias of a duckdb_logical_type, if one is set, else `NULL` -You must free the result. +Returns the alias of a duckdb_logical_type, if one is set, else `NULL`. +The result must be destroyed with `duckdb_free`. * type: The logical type to return the alias of * returns: The alias or `NULL` @@ -1472,14 +1673,14 @@ The resulting type should be destroyed with `duckdb_destroy_logical_type`. DUCKDB_API duckdb_logical_type duckdb_create_map_type(duckdb_logical_type key_type, duckdb_logical_type value_type); /*! -Creates a UNION type from the passed types array +Creates a UNION type from the passed types array. The resulting type should be destroyed with `duckdb_destroy_logical_type`. * types: The array of types that the union should consist of. * type_amount: The size of the types array. * returns: The logical type. */ -DUCKDB_API duckdb_logical_type duckdb_create_union_type(duckdb_logical_type member_types, const char **member_names, +DUCKDB_API duckdb_logical_type duckdb_create_union_type(duckdb_logical_type *member_types, const char **member_names, idx_t member_count); /*! @@ -1506,7 +1707,7 @@ The resulting type should be destroyed with `duckdb_destroy_logical_type`. DUCKDB_API duckdb_logical_type duckdb_create_enum_type(const char **member_names, idx_t member_count); /*! -Creates a `duckdb_logical_type` of type decimal with the specified width and scale +Creates a `duckdb_logical_type` of type decimal with the specified width and scale. The resulting type should be destroyed with `duckdb_destroy_logical_type`. * width: The width of the decimal type @@ -1516,7 +1717,7 @@ The resulting type should be destroyed with `duckdb_destroy_logical_type`. DUCKDB_API duckdb_logical_type duckdb_create_decimal_type(uint8_t width, uint8_t scale); /*! -Retrieves the type class of a `duckdb_logical_type`. +Retrieves the enum type class of a `duckdb_logical_type`. * type: The logical type object * returns: The type id @@ -1556,7 +1757,7 @@ Retrieves the internal storage type of an enum type. DUCKDB_API duckdb_type duckdb_enum_internal_type(duckdb_logical_type type); /*! -Retrieves the dictionary size of the enum type +Retrieves the dictionary size of the enum type. * type: The logical type object * returns: The dictionary size of the enum type @@ -1566,7 +1767,7 @@ DUCKDB_API uint32_t duckdb_enum_dictionary_size(duckdb_logical_type type); /*! Retrieves the dictionary value at the specified position from the enum. -The result must be freed with `duckdb_free` +The result must be freed with `duckdb_free`. * type: The logical type object * index: The index in the dictionary @@ -1577,7 +1778,7 @@ DUCKDB_API char *duckdb_enum_dictionary_value(duckdb_logical_type type, idx_t in /*! Retrieves the child type of the given list type. -The result must be freed with `duckdb_destroy_logical_type` +The result must be freed with `duckdb_destroy_logical_type`. * type: The logical type object * returns: The child type of the list type. Must be destroyed with `duckdb_destroy_logical_type`. @@ -1587,7 +1788,7 @@ DUCKDB_API duckdb_logical_type duckdb_list_type_child_type(duckdb_logical_type t /*! Retrieves the key type of the given map type. -The result must be freed with `duckdb_destroy_logical_type` +The result must be freed with `duckdb_destroy_logical_type`. * type: The logical type object * returns: The key type of the map type. Must be destroyed with `duckdb_destroy_logical_type`. @@ -1597,7 +1798,7 @@ DUCKDB_API duckdb_logical_type duckdb_map_type_key_type(duckdb_logical_type type /*! Retrieves the value type of the given map type. -The result must be freed with `duckdb_destroy_logical_type` +The result must be freed with `duckdb_destroy_logical_type`. * type: The logical type object * returns: The value type of the map type. Must be destroyed with `duckdb_destroy_logical_type`. @@ -1615,7 +1816,7 @@ DUCKDB_API idx_t duckdb_struct_type_child_count(duckdb_logical_type type); /*! Retrieves the name of the struct child. -The result must be freed with `duckdb_free` +The result must be freed with `duckdb_free`. * type: The logical type object * index: The child index @@ -1626,7 +1827,7 @@ DUCKDB_API char *duckdb_struct_type_child_name(duckdb_logical_type type, idx_t i /*! Retrieves the child type of the given struct type at the specified index. -The result must be freed with `duckdb_destroy_logical_type` +The result must be freed with `duckdb_destroy_logical_type`. * type: The logical type object * index: The child index @@ -1645,7 +1846,7 @@ DUCKDB_API idx_t duckdb_union_type_member_count(duckdb_logical_type type); /*! Retrieves the name of the union member. -The result must be freed with `duckdb_free` +The result must be freed with `duckdb_free`. * type: The logical type object * index: The child index @@ -1656,7 +1857,7 @@ DUCKDB_API char *duckdb_union_type_member_name(duckdb_logical_type type, idx_t i /*! Retrieves the child type of the given union member at the specified index. -The result must be freed with `duckdb_destroy_logical_type` +The result must be freed with `duckdb_destroy_logical_type`. * type: The logical type object * index: The child index @@ -1674,9 +1875,12 @@ DUCKDB_API void duckdb_destroy_logical_type(duckdb_logical_type *type); //===--------------------------------------------------------------------===// // Data Chunk Interface //===--------------------------------------------------------------------===// + /*! Creates an empty DataChunk with the specified set of types. +Note that the result must be destroyed with `duckdb_destroy_data_chunk`. + * types: An array of types of the data chunk. * column_count: The number of columns. * returns: The data chunk. @@ -1735,6 +1939,7 @@ DUCKDB_API void duckdb_data_chunk_set_size(duckdb_data_chunk chunk, idx_t size); //===--------------------------------------------------------------------===// // Vector Interface //===--------------------------------------------------------------------===// + /*! Retrieves the column type of the specified vector. @@ -1819,7 +2024,7 @@ The resulting vector is valid as long as the parent vector is valid. DUCKDB_API duckdb_vector duckdb_list_vector_get_child(duckdb_vector vector); /*! -Returns the size of the child vector of the list +Returns the size of the child vector of the list. * vector: The vector * returns: The size of the child list @@ -1858,6 +2063,7 @@ DUCKDB_API duckdb_vector duckdb_struct_vector_get_child(duckdb_vector vector, id //===--------------------------------------------------------------------===// // Validity Mask Functions //===--------------------------------------------------------------------===// + /*! Returns whether or not a row is valid (i.e. not NULL) in the given validity mask. @@ -1902,15 +2108,6 @@ DUCKDB_API void duckdb_validity_set_row_valid(uint64_t *validity, idx_t row); //===--------------------------------------------------------------------===// // Table Functions //===--------------------------------------------------------------------===// -typedef void *duckdb_table_function; -typedef void *duckdb_bind_info; -typedef void *duckdb_init_info; -typedef void *duckdb_function_info; - -typedef void (*duckdb_table_function_bind_t)(duckdb_bind_info info); -typedef void (*duckdb_table_function_init_t)(duckdb_init_info info); -typedef void (*duckdb_table_function_t)(duckdb_function_info info, duckdb_data_chunk output); -typedef void (*duckdb_delete_callback_t)(void *data); /*! Creates a new empty table function. @@ -1965,7 +2162,7 @@ DUCKDB_API void duckdb_table_function_set_extra_info(duckdb_table_function table duckdb_delete_callback_t destroy); /*! -Sets the bind function of the table function +Sets the bind function of the table function. * table_function: The table function * bind: The bind function @@ -1973,7 +2170,7 @@ Sets the bind function of the table function DUCKDB_API void duckdb_table_function_set_bind(duckdb_table_function table_function, duckdb_table_function_bind_t bind); /*! -Sets the init function of the table function +Sets the init function of the table function. * table_function: The table function * init: The init function @@ -1981,7 +2178,7 @@ Sets the init function of the table function DUCKDB_API void duckdb_table_function_set_init(duckdb_table_function table_function, duckdb_table_function_init_t init); /*! -Sets the thread-local init function of the table function +Sets the thread-local init function of the table function. * table_function: The table function * init: The init function @@ -1990,7 +2187,7 @@ DUCKDB_API void duckdb_table_function_set_local_init(duckdb_table_function table duckdb_table_function_init_t init); /*! -Sets the main function of the table function +Sets the main function of the table function. * table_function: The table function * function: The function @@ -2026,8 +2223,9 @@ DUCKDB_API duckdb_state duckdb_register_table_function(duckdb_connection con, du //===--------------------------------------------------------------------===// // Table Function Bind //===--------------------------------------------------------------------===// + /*! -Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info` +Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info`. * info: The info object * returns: The extra info @@ -2103,7 +2301,7 @@ DUCKDB_API void duckdb_bind_set_error(duckdb_bind_info info, const char *error); //===--------------------------------------------------------------------===// /*! -Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info` +Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info`. * info: The info object * returns: The extra info @@ -2172,12 +2370,13 @@ DUCKDB_API void duckdb_init_set_error(duckdb_init_info info, const char *error); //===--------------------------------------------------------------------===// /*! -Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info` +Retrieves the extra info of the function as set in `duckdb_table_function_set_extra_info`. * info: The info object * returns: The extra info */ DUCKDB_API void *duckdb_function_get_extra_info(duckdb_function_info info); + /*! Gets the bind data set by `duckdb_bind_set_bind_data` during the bind. @@ -2216,12 +2415,9 @@ DUCKDB_API void duckdb_function_set_error(duckdb_function_info info, const char //===--------------------------------------------------------------------===// // Replacement Scans //===--------------------------------------------------------------------===// -typedef void *duckdb_replacement_scan_info; - -typedef void (*duckdb_replacement_callback_t)(duckdb_replacement_scan_info info, const char *table_name, void *data); /*! -Add a replacement scan definition to the specified database +Add a replacement scan definition to the specified database. * db: The database object to add the replacement scan to * replacement: The replacement scan callback @@ -2232,8 +2428,8 @@ DUCKDB_API void duckdb_add_replacement_scan(duckdb_database db, duckdb_replaceme void *extra_data, duckdb_delete_callback_t delete_callback); /*! -Sets the replacement function name to use. If this function is called in the replacement callback, - the replacement scan is performed. If it is not called, the replacement callback is not performed. +Sets the replacement function name. If this function is called in the replacement callback, +the replacement scan is performed. If it is not called, the replacement callback is not performed. * info: The info object * function_name: The function name to substitute. @@ -2267,12 +2463,17 @@ DUCKDB_API void duckdb_replacement_scan_set_error(duckdb_replacement_scan_info i // the row should be finished by calling `duckdb_appender_end_row`. After all rows have been appended, // `duckdb_appender_destroy` should be used to finalize the appender and clean up the resulting memory. +// Instead of appending rows with `duckdb_appender_end_row`, it is also possible to fill and append +// chunks-at-a-time. + // Note that `duckdb_appender_destroy` should always be called on the resulting appender, even if the function returns // `DuckDBError`. /*! Creates an appender object. +Note that the object must be destroyed with `duckdb_appender_destroy`. + * connection: The connection context to create the appender in. * schema: The schema of the table to append to, or `nullptr` for the default schema. * table: The table name to append to. @@ -2282,6 +2483,25 @@ Creates an appender object. DUCKDB_API duckdb_state duckdb_appender_create(duckdb_connection connection, const char *schema, const char *table, duckdb_appender *out_appender); +/*! +Returns the number of columns in the table that belongs to the appender. + +* appender The appender to get the column count from. +* returns: The number of columns in the table. +*/ +DUCKDB_API idx_t duckdb_appender_column_count(duckdb_appender appender); + +/*! +Returns the type of the column at the specified index. + +Note: The resulting type should be destroyed with `duckdb_destroy_logical_type`. + +* appender The appender to get the column type from. +* col_idx The index of the column to get the type of. +* returns: The duckdb_logical_type of the column. +*/ +DUCKDB_API duckdb_logical_type duckdb_appender_column_type(duckdb_appender appender, idx_t col_idx); + /*! Returns the error message associated with the given appender. If the appender has no error message, this returns `nullptr` instead. @@ -2346,18 +2566,22 @@ DUCKDB_API duckdb_state duckdb_append_bool(duckdb_appender appender, bool value) Append an int8_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_int8(duckdb_appender appender, int8_t value); + /*! Append an int16_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_int16(duckdb_appender appender, int16_t value); + /*! Append an int32_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_int32(duckdb_appender appender, int32_t value); + /*! Append an int64_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_int64(duckdb_appender appender, int64_t value); + /*! Append a duckdb_hugeint value to the appender. */ @@ -2367,23 +2591,32 @@ DUCKDB_API duckdb_state duckdb_append_hugeint(duckdb_appender appender, duckdb_h Append a uint8_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_uint8(duckdb_appender appender, uint8_t value); + /*! Append a uint16_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_uint16(duckdb_appender appender, uint16_t value); + /*! Append a uint32_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_uint32(duckdb_appender appender, uint32_t value); + /*! Append a uint64_t value to the appender. */ DUCKDB_API duckdb_state duckdb_append_uint64(duckdb_appender appender, uint64_t value); +/*! +Append a duckdb_uhugeint value to the appender. +*/ +DUCKDB_API duckdb_state duckdb_append_uhugeint(duckdb_appender appender, duckdb_uhugeint value); + /*! Append a float value to the appender. */ DUCKDB_API duckdb_state duckdb_append_float(duckdb_appender appender, float value); + /*! Append a double value to the appender. */ @@ -2393,14 +2626,17 @@ DUCKDB_API duckdb_state duckdb_append_double(duckdb_appender appender, double va Append a duckdb_date value to the appender. */ DUCKDB_API duckdb_state duckdb_append_date(duckdb_appender appender, duckdb_date value); + /*! Append a duckdb_time value to the appender. */ DUCKDB_API duckdb_state duckdb_append_time(duckdb_appender appender, duckdb_time value); + /*! Append a duckdb_timestamp value to the appender. */ DUCKDB_API duckdb_state duckdb_append_timestamp(duckdb_appender appender, duckdb_timestamp value); + /*! Append a duckdb_interval value to the appender. */ @@ -2410,14 +2646,17 @@ DUCKDB_API duckdb_state duckdb_append_interval(duckdb_appender appender, duckdb_ Append a varchar value to the appender. */ DUCKDB_API duckdb_state duckdb_append_varchar(duckdb_appender appender, const char *val); + /*! Append a varchar value to the appender. */ DUCKDB_API duckdb_state duckdb_append_varchar_length(duckdb_appender appender, const char *val, idx_t length); + /*! Append a blob value to the appender. */ DUCKDB_API duckdb_state duckdb_append_blob(duckdb_appender appender, const void *data, idx_t length); + /*! Append a NULL value to the appender (of any type). */ @@ -2439,6 +2678,7 @@ DUCKDB_API duckdb_state duckdb_append_data_chunk(duckdb_appender appender, duckd //===--------------------------------------------------------------------===// // Arrow Interface //===--------------------------------------------------------------------===// + /*! Executes a SQL query within a connection and stores the full (materialized) result in an arrow structure. If the query fails to execute, DuckDBError is returned and the error message can be retrieved by calling @@ -2455,7 +2695,8 @@ query fails, otherwise the error stored within the result will not be freed corr DUCKDB_API duckdb_state duckdb_query_arrow(duckdb_connection connection, const char *query, duckdb_arrow *out_result); /*! -Fetch the internal arrow schema from the arrow result. +Fetch the internal arrow schema from the arrow result. Remember to call release on the respective +ArrowSchema object. * result: The result to fetch the schema from. * out_schema: The output schema. @@ -2464,7 +2705,8 @@ Fetch the internal arrow schema from the arrow result. DUCKDB_API duckdb_state duckdb_query_arrow_schema(duckdb_arrow result, duckdb_arrow_schema *out_schema); /*! -Fetch the internal arrow schema from the prepared statement. +Fetch the internal arrow schema from the prepared statement. Remember to call release on the respective +ArrowSchema object. * result: The prepared statement to fetch the schema from. * out_schema: The output schema. @@ -2473,7 +2715,8 @@ 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. +Convert a data chunk into an arrow struct array. Remember to call release on the respective +ArrowArray object. * result: The result object the data chunk have been fetched from. * chunk: The data chunk to convert. @@ -2482,7 +2725,8 @@ Convert a data chunk into an arrow struct array. DUCKDB_API void duckdb_result_arrow_array(duckdb_result result, duckdb_data_chunk chunk, duckdb_arrow_array *out_array); /*! -Fetch an internal arrow struct array from the arrow result. +Fetch an internal arrow struct array from the arrow result. Remember to call release on the respective +ArrowArray object. 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. @@ -2494,7 +2738,7 @@ So consume the out_array before calling this function again. DUCKDB_API duckdb_state duckdb_query_arrow_array(duckdb_arrow result, duckdb_arrow_array *out_array); /*! -Returns the number of columns present in a the arrow result object. +Returns the number of columns present in the arrow result object. * result: The result object. * returns: The number of columns present in the result object. @@ -2502,7 +2746,7 @@ Returns the number of columns present in a the arrow result object. DUCKDB_API idx_t duckdb_arrow_column_count(duckdb_arrow result); /*! -Returns the number of rows present in a the arrow result object. +Returns the number of rows present in the arrow result object. * result: The result object. * returns: The number of rows present in the result object. @@ -2524,7 +2768,7 @@ Returns the error message contained within the result. The error is only set if The error message should not be freed. It will be de-allocated when `duckdb_destroy_arrow` is called. -* result: The result object to fetch the nullmask from. +* result: The result object to fetch the error from. * returns: The error of the result. */ DUCKDB_API const char *duckdb_query_arrow_error(duckdb_arrow result); @@ -2536,10 +2780,53 @@ Closes the result and de-allocates all memory allocated for the arrow result. */ DUCKDB_API void duckdb_destroy_arrow(duckdb_arrow *result); +/*! +Releases the arrow array stream and de-allocates its memory. + +* stream: The arrow array stream to destroy. +*/ +DUCKDB_API void duckdb_destroy_arrow_stream(duckdb_arrow_stream *stream_p); + +/*! +Executes the prepared statement with the given bound parameters, and returns an arrow query result. +Note that after running `duckdb_execute_prepared_arrow`, `duckdb_destroy_arrow` must be called on the result object. + +* prepared_statement: The prepared statement to execute. +* out_result: The query result. +* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. +*/ +DUCKDB_API duckdb_state duckdb_execute_prepared_arrow(duckdb_prepared_statement prepared_statement, + duckdb_arrow *out_result); + +/*! +Scans the Arrow stream and creates a view with the given name. + +* connection: The connection on which to execute the scan. +* table_name: Name of the temporary view to create. +* arrow: Arrow stream wrapper. +* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. +*/ +DUCKDB_API duckdb_state duckdb_arrow_scan(duckdb_connection connection, const char *table_name, + duckdb_arrow_stream arrow); + +/*! +Scans the Arrow array and creates a view with the given name. +Note that after running `duckdb_arrow_array_scan`, `duckdb_destroy_arrow_stream` must be called on the out stream. + +* connection: The connection on which to execute the scan. +* table_name: Name of the temporary view to create. +* arrow_schema: Arrow schema wrapper. +* arrow_array: Arrow array wrapper. +* out_stream: Output array stream that wraps around the passed schema, for releasing/deleting once done. +* returns: `DuckDBSuccess` on success or `DuckDBError` on failure. +*/ +DUCKDB_API duckdb_state duckdb_arrow_array_scan(duckdb_connection connection, const char *table_name, + duckdb_arrow_schema arrow_schema, duckdb_arrow_array arrow_array, + duckdb_arrow_stream *out_stream); + //===--------------------------------------------------------------------===// // Threading Information //===--------------------------------------------------------------------===// -typedef void *duckdb_task_state; /*! Execute DuckDB tasks on this thread. @@ -2553,9 +2840,9 @@ DUCKDB_API void duckdb_execute_tasks(duckdb_database database, idx_t max_tasks); /*! Creates a task state that can be used with duckdb_execute_tasks_state to execute tasks until - duckdb_finish_execution is called on the state. +`duckdb_finish_execution` is called on the state. -duckdb_destroy_state should be called on the result in order to free memory. +`duckdb_destroy_state` must be called on the result. * database: The database object to create the task state for * returns: The task state that can be used with duckdb_execute_tasks_state. @@ -2612,7 +2899,7 @@ on the task state. DUCKDB_API void duckdb_destroy_task_state(duckdb_task_state state); /*! -Returns true if execution of the current query is finished. +Returns true if the execution of the current query is finished. * con: The connection on which to check */ diff --git a/src/duckdb/src/include/duckdb/catalog/catalog.hpp b/src/duckdb/src/include/duckdb/catalog/catalog.hpp index f04f027be..988d54c74 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog.hpp @@ -16,6 +16,7 @@ #include "duckdb/common/atomic.hpp" #include "duckdb/common/optional_ptr.hpp" #include "duckdb/common/enums/on_entry_not_found.hpp" +#include "duckdb/common/exception/catalog_exception.hpp" #include namespace duckdb { @@ -238,7 +239,7 @@ class Catalog { return nullptr; } if (entry->type != T::Type) { - throw CatalogException(error_context.FormatError("%s is not an %s", name, T::Name)); + throw CatalogException(error_context, "%s is not an %s", name, T::Name); } return &entry->template Cast(); } @@ -282,7 +283,7 @@ class Catalog { return nullptr; } if (entry->type != T::Type) { - throw CatalogException(error_context.FormatError("%s is not an %s", name, T::Name)); + throw CatalogException(error_context, "%s is not an %s", name, T::Name); } return &entry->template Cast(); } @@ -306,7 +307,7 @@ class Catalog { //! Autoload the extension required for `configuration_name` or throw a CatalogException static void AutoloadExtensionByConfigName(ClientContext &context, const string &configuration_name); //! Autoload the extension required for `function_name` or throw a CatalogException - static bool AutoLoadExtensionByCatalogEntry(ClientContext &context, CatalogType type, const string &entry_name); + static bool AutoLoadExtensionByCatalogEntry(DatabaseInstance &db, CatalogType type, const string &entry_name); DUCKDB_API static bool TryAutoLoad(ClientContext &context, const string &extension_name) noexcept; protected: diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry.hpp index 50d001d6f..995b8d00d 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry.hpp @@ -13,6 +13,8 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/atomic.hpp" #include "duckdb/common/optional_ptr.hpp" +#include "duckdb/common/exception/catalog_exception.hpp" +#include "duckdb/common/types/value.hpp" #include namespace duckdb { @@ -23,6 +25,7 @@ class ClientContext; class SchemaCatalogEntry; class Serializer; class Deserializer; +class Value; struct CreateInfo; @@ -49,6 +52,8 @@ class CatalogEntry { bool internal; //! Timestamp at which the catalog entry was created atomic timestamp; + //! (optional) comment on this entry + Value comment; private: //! Child entry diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp index 2517e154f..44923729a 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_index_entry.hpp @@ -12,15 +12,27 @@ namespace duckdb { +//! Wrapper class to allow copying a DuckIndexEntry (for altering the DuckIndexEntry metadata such as comments) +struct IndexDataTableInfo { + IndexDataTableInfo(shared_ptr &info_p, const string &index_name_p); + ~IndexDataTableInfo(); + + //! Pointer to the DataTableInfo + shared_ptr info; + //! The index to be removed on destruction + string index_name; +}; + //! A duck index entry class DuckIndexEntry : public IndexCatalogEntry { public: //! Create a DuckIndexEntry DuckIndexEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateIndexInfo &info); - ~DuckIndexEntry() override; + + virtual unique_ptr Copy(ClientContext &context) const override; //! The indexed table information - shared_ptr info; + shared_ptr info; //! We need the initial size of the index after the CREATE INDEX statement, //! as it is necessary to determine the auto checkpoint threshold @@ -30,6 +42,8 @@ class DuckIndexEntry : public IndexCatalogEntry { string GetSchemaName() const override; string GetTableName() const override; + DataTableInfo &GetDataTableInfo() const; + //! Drops in-memory index data and marks all blocks on disk as free blocks, allowing to reclaim them void CommitDrop(); }; diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_schema_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_schema_entry.hpp index b1dea57be..01ef6fcf7 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_schema_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_schema_entry.hpp @@ -15,7 +15,7 @@ namespace duckdb { //! A schema in the catalog class DuckSchemaEntry : public SchemaCatalogEntry { public: - DuckSchemaEntry(Catalog &catalog, string name, bool is_internal); + DuckSchemaEntry(Catalog &catalog, CreateSchemaInfo &info); private: //! The catalog set holding the tables @@ -64,6 +64,8 @@ class DuckSchemaEntry : public SchemaCatalogEntry { optional_ptr GetEntry(CatalogTransaction transaction, CatalogType type, const string &name) override; SimilarCatalogEntry GetSimilarEntry(CatalogTransaction transaction, CatalogType type, const string &name) override; + unique_ptr Copy(ClientContext &context) const override; + void Verify(Catalog &catalog) override; private: diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_table_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_table_entry.hpp index 06d166d16..0890ce829 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_table_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/duck_table_entry.hpp @@ -57,6 +57,7 @@ class DuckTableEntry : public TableCatalogEntry { unique_ptr DropNotNull(ClientContext &context, DropNotNullInfo &info); unique_ptr AddForeignKeyConstraint(ClientContext &context, AlterForeignKeyInfo &info); unique_ptr DropForeignKeyConstraint(ClientContext &context, AlterForeignKeyInfo &info); + unique_ptr SetColumnComment(ClientContext &context, SetColumnCommentInfo &info); void UpdateConstraintsOnColumnDrop(const LogicalIndex &removed_index, const vector &adjusted_indices, const RemoveColumnInfo &info, CreateTableInfo &create_info, bool is_generated); diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/scalar_macro_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/scalar_macro_catalog_entry.hpp index 36ec09b70..736cb9ca8 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/scalar_macro_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/scalar_macro_catalog_entry.hpp @@ -23,5 +23,7 @@ class ScalarMacroCatalogEntry : public MacroCatalogEntry { public: ScalarMacroCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateMacroInfo &info); + + unique_ptr Copy(ClientContext &context) const override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/schema_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/schema_catalog_entry.hpp index ead175709..0356ea308 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/schema_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/schema_catalog_entry.hpp @@ -44,7 +44,7 @@ class SchemaCatalogEntry : public InCatalogEntry { static constexpr const char *Name = "schema"; public: - SchemaCatalogEntry(Catalog &catalog, string name, bool is_internal); + SchemaCatalogEntry(Catalog &catalog, CreateSchemaInfo &info); public: unique_ptr GetInfo() const override; diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp index 631df9887..f96e6b9ea 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp @@ -14,6 +14,7 @@ #include "duckdb/parser/parsed_data/alter_table_info.hpp" namespace duckdb { +class DuckTransaction; struct SequenceValue { SequenceValue() : usage_count(0), counter(-1) { @@ -25,18 +26,9 @@ struct SequenceValue { int64_t counter; }; -//! A sequence catalog entry -class SequenceCatalogEntry : public StandardEntry { -public: - static constexpr const CatalogType Type = CatalogType::SEQUENCE_ENTRY; - static constexpr const char *Name = "sequence"; - -public: - //! Create a real TableCatalogEntry and initialize storage for it - SequenceCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateSequenceInfo &info); +struct SequenceData { + explicit SequenceData(CreateSequenceInfo &info); - //! Lock for getting a value on the sequence - mutex lock; //! The amount of times the sequence has been used uint64_t usage_count; //! The sequence counter @@ -53,10 +45,33 @@ class SequenceCatalogEntry : public StandardEntry { int64_t max_value; //! Whether or not the sequence cycles bool cycle; +}; + +//! A sequence catalog entry +class SequenceCatalogEntry : public StandardEntry { +public: + static constexpr const CatalogType Type = CatalogType::SEQUENCE_ENTRY; + static constexpr const char *Name = "sequence"; + +public: + //! Create a real TableCatalogEntry and initialize storage for it + SequenceCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateSequenceInfo &info); public: + virtual unique_ptr Copy(ClientContext &context) const override; unique_ptr GetInfo() const override; + SequenceData GetData() const; + int64_t CurrentValue(); + int64_t NextValue(DuckTransaction &transaction); + void ReplayValue(uint64_t usage_count, int64_t counter); + string ToSQL() const override; + +private: + //! Lock for getting a value on the sequence + mutable mutex lock; + //! Sequence data + SequenceData data; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_catalog_entry.hpp index 207e52e73..243765a45 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_catalog_entry.hpp @@ -32,6 +32,7 @@ struct ChangeColumnTypeInfo; struct AlterForeignKeyInfo; struct SetNotNullInfo; struct DropNotNullInfo; +struct SetColumnCommentInfo; class TableFunction; struct FunctionData; diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_macro_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_macro_catalog_entry.hpp index 171d14e59..bbea06a7c 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_macro_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/table_macro_catalog_entry.hpp @@ -22,6 +22,8 @@ class TableMacroCatalogEntry : public MacroCatalogEntry { public: TableMacroCatalogEntry(Catalog &catalog, SchemaCatalogEntry &schema, CreateMacroInfo &info); + + unique_ptr Copy(ClientContext &context) const override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_entry/type_catalog_entry.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_entry/type_catalog_entry.hpp index c5f61cfbf..97fdce21a 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_entry/type_catalog_entry.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_entry/type_catalog_entry.hpp @@ -28,6 +28,7 @@ class TypeCatalogEntry : public StandardEntry { public: unique_ptr GetInfo() const override; + unique_ptr Copy(ClientContext &context) const override; string ToSQL() const override; }; diff --git a/src/duckdb/src/include/duckdb/catalog/catalog_transaction.hpp b/src/duckdb/src/include/duckdb/catalog/catalog_transaction.hpp index 47fa68a7d..f9a327e7a 100644 --- a/src/duckdb/src/include/duckdb/catalog/catalog_transaction.hpp +++ b/src/duckdb/src/include/duckdb/catalog/catalog_transaction.hpp @@ -29,6 +29,7 @@ struct CatalogTransaction { ClientContext &GetContext(); + static CatalogTransaction GetSystemCatalogTransaction(ClientContext &context); static CatalogTransaction GetSystemTransaction(DatabaseInstance &db); }; diff --git a/src/duckdb/src/include/duckdb/catalog/default/builtin_types/types.hpp b/src/duckdb/src/include/duckdb/catalog/default/builtin_types/types.hpp index 817aefbc4..1712b8654 100644 --- a/src/duckdb/src/include/duckdb/catalog/default/builtin_types/types.hpp +++ b/src/duckdb/src/include/duckdb/catalog/default/builtin_types/types.hpp @@ -19,7 +19,7 @@ struct DefaultType { LogicalTypeId type; }; -using builtin_type_array = std::array; +using builtin_type_array = std::array; static constexpr const builtin_type_array BUILTIN_TYPES{{ {"decimal", LogicalTypeId::DECIMAL}, @@ -48,6 +48,8 @@ static constexpr const builtin_type_array BUILTIN_TYPES{{ {"binary", LogicalTypeId::BLOB}, {"hugeint", LogicalTypeId::HUGEINT}, {"int128", LogicalTypeId::HUGEINT}, + {"uhugeint", LogicalTypeId::UHUGEINT}, + {"uint128", LogicalTypeId::UHUGEINT}, {"bigint", LogicalTypeId::BIGINT}, {"oid", LogicalTypeId::BIGINT}, {"long", LogicalTypeId::BIGINT}, diff --git a/src/duckdb/src/include/duckdb/common/arrow/arrow.hpp b/src/duckdb/src/include/duckdb/common/arrow/arrow.hpp index b2f613ec8..49bcb1d8b 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/arrow.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/arrow.hpp @@ -22,7 +22,7 @@ extern "C" { #define ARROW_FLAG_MAP_KEYS_SORTED 4 struct ArrowSchema { - // Array type description + //! Array type description const char *format; const char *name; const char *metadata; @@ -31,14 +31,22 @@ struct ArrowSchema { struct ArrowSchema **children; struct ArrowSchema *dictionary; - // Release callback + //! Release callback void (*release)(struct ArrowSchema *); - // Opaque producer-specific data + //! Opaque producer-specific data void *private_data; + + //! Initialize all fields + void Init() { + flags = 0; + n_children = 0; + release = nullptr; + private_data = nullptr; + } }; struct ArrowArray { - // Array data description + //! Array data description int64_t length; int64_t null_count; int64_t offset; @@ -48,10 +56,21 @@ struct ArrowArray { struct ArrowArray **children; struct ArrowArray *dictionary; - // Release callback + //! Release callback void (*release)(struct ArrowArray *); - // Opaque producer-specific data + //! Opaque producer-specific data void *private_data; + + //! Initialize all fields + void Init() { + length = 0; + null_count = 0; + offset = 0; + n_buffers = 0; + n_children = 0; + release = nullptr; + private_data = nullptr; + } }; #endif diff --git a/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp b/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp index 0d3fc60db..8b6511564 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/arrow_wrapper.hpp @@ -9,7 +9,7 @@ #pragma once #include "duckdb/common/arrow/arrow.hpp" #include "duckdb/common/helper.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/main/chunk_scan_state.hpp" #include "duckdb/main/client_properties.hpp" @@ -62,10 +62,10 @@ class ArrowArrayStreamWrapper { class ArrowUtil { public: static bool TryFetchChunk(ChunkScanState &scan_state, ClientProperties options, idx_t chunk_size, ArrowArray *out, - idx_t &result_count, PreservedError &error); + idx_t &result_count, ErrorData &error); static idx_t FetchChunk(ChunkScanState &scan_state, ClientProperties options, idx_t chunk_size, ArrowArray *out); private: - static bool TryFetchNext(QueryResult &result, unique_ptr &out, PreservedError &error); + static bool TryFetchNext(QueryResult &result, unique_ptr &out, ErrorData &error); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/arrow/result_arrow_wrapper.hpp b/src/duckdb/src/include/duckdb/common/arrow/result_arrow_wrapper.hpp index 629316de9..f82e472dd 100644 --- a/src/duckdb/src/include/duckdb/common/arrow/result_arrow_wrapper.hpp +++ b/src/duckdb/src/include/duckdb/common/arrow/result_arrow_wrapper.hpp @@ -20,7 +20,7 @@ class ResultArrowArrayStreamWrapper { public: ArrowArrayStream stream; unique_ptr result; - PreservedError last_error; + ErrorData last_error; idx_t batch_size; vector column_types; vector column_names; diff --git a/src/duckdb/src/include/duckdb/common/bit_utils.hpp b/src/duckdb/src/include/duckdb/common/bit_utils.hpp index da4aad663..28bc4a975 100644 --- a/src/duckdb/src/include/duckdb/common/bit_utils.hpp +++ b/src/duckdb/src/include/duckdb/common/bit_utils.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/hugeint.hpp" +#include "duckdb/common/uhugeint.hpp" #ifdef _MSC_VER #define __restrict__ @@ -138,4 +139,33 @@ struct CountZeros { } }; +template <> +struct CountZeros { + inline static int Leading(uhugeint_t value) { + const uint64_t upper = (uint64_t)value.upper; + const uint64_t lower = value.lower; + + if (upper) { + return __builtin_clzll(upper); + } else if (lower) { + return 64 + __builtin_clzll(lower); + } else { + return 128; + } + } + + inline static int Trailing(uhugeint_t value) { + const uint64_t upper = (uint64_t)value.upper; + const uint64_t lower = value.lower; + + if (lower) { + return __builtin_ctzll(lower); + } else if (upper) { + return 64 + __builtin_ctzll(upper); + } else { + return 128; + } + } +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/bitpacking.hpp b/src/duckdb/src/include/duckdb/common/bitpacking.hpp index 43a20042b..d41a97a21 100644 --- a/src/duckdb/src/include/duckdb/common/bitpacking.hpp +++ b/src/duckdb/src/include/duckdb/common/bitpacking.hpp @@ -20,8 +20,8 @@ namespace duckdb { using bitpacking_width_t = uint8_t; struct HugeIntPacker { - static void Pack(const hugeint_t *__restrict in, uint32_t *__restrict out, bitpacking_width_t width); - static void Unpack(const uint32_t *__restrict in, hugeint_t *__restrict out, bitpacking_width_t width); + static void Pack(const uhugeint_t *__restrict in, uint32_t *__restrict out, bitpacking_width_t width); + static void Unpack(const uint32_t *__restrict in, uhugeint_t *__restrict out, bitpacking_width_t width); }; class BitpackingPrimitives { @@ -223,8 +223,8 @@ class BitpackingPrimitives { } else if (std::is_same::value || std::is_same::value) { duckdb_fastpforlib::fastpack(reinterpret_cast(values), reinterpret_cast(dst), static_cast(width)); - } else if (std::is_same::value) { - HugeIntPacker::Pack(reinterpret_cast(values), reinterpret_cast(dst), width); + } else if (std::is_same::value || std::is_same::value) { + HugeIntPacker::Pack(reinterpret_cast(values), reinterpret_cast(dst), width); } else { throw InternalException("Unsupported type for bitpacking"); } @@ -245,8 +245,8 @@ class BitpackingPrimitives { } else if (std::is_same::value || std::is_same::value) { duckdb_fastpforlib::fastunpack(reinterpret_cast(src), reinterpret_cast(dst), static_cast(width)); - } else if (std::is_same::value) { - HugeIntPacker::Unpack(reinterpret_cast(src), reinterpret_cast(dst), width); + } else if (std::is_same::value || std::is_same::value) { + HugeIntPacker::Unpack(reinterpret_cast(src), reinterpret_cast(dst), width); } else { throw InternalException("Unsupported type for bitpacking"); } diff --git a/src/duckdb/src/include/duckdb/common/enum_util.hpp b/src/duckdb/src/include/duckdb/common/enum_util.hpp index b38e1c1aa..9587715c2 100644 --- a/src/duckdb/src/include/duckdb/common/enum_util.hpp +++ b/src/duckdb/src/include/duckdb/common/enum_util.hpp @@ -34,6 +34,8 @@ struct EnumUtil { enum class AccessMode : uint8_t; +enum class AggregateCombineType : uint8_t; + enum class AggregateHandling : uint8_t; enum class AggregateOrderDependent : uint8_t; @@ -100,6 +102,8 @@ enum class DefaultOrderByNullType : uint8_t; enum class DependencyEntryType : uint8_t; +enum class DeprecatedIndexType : uint8_t; + enum class DistinctType : uint8_t; enum class ErrorType : uint16_t; @@ -116,6 +120,8 @@ enum class ExpressionType : uint8_t; enum class ExtensionLoadResult : uint8_t; +enum class ExtraDropInfoType : uint8_t; + enum class ExtraTypeInfoType : uint8_t; enum class FileBufferType : uint8_t; @@ -132,14 +138,12 @@ enum class ForeignKeyType : uint8_t; enum class FunctionNullHandling : uint8_t; -enum class FunctionSideEffects : uint8_t; +enum class FunctionStability : uint8_t; enum class HLLStorageType : uint8_t; enum class IndexConstraintType : uint8_t; -enum class IndexType : uint8_t; - enum class InsertColumnOrder : uint8_t; enum class InterruptMode : uint8_t; @@ -190,8 +194,6 @@ enum class ParseInfoType : uint8_t; enum class ParserExtensionResultType : uint8_t; -enum class ParserMode : uint8_t; - enum class PartitionSortStage : uint8_t; enum class PartitionedColumnDataType : uint8_t; @@ -226,6 +228,10 @@ enum class ResultModifierType : uint8_t; enum class SampleMethod : uint8_t; +enum class SecretDisplayType : uint8_t; + +enum class SecretPersistType : uint8_t; + enum class SequenceInfo : uint8_t; enum class SetOperationType : uint8_t; @@ -234,6 +240,8 @@ enum class SetScope : uint8_t; enum class SetType : uint8_t; +enum class ShowType : uint8_t; + enum class SimplifiedTokenType : uint8_t; enum class SinkCombineResultType : uint8_t; @@ -304,6 +312,9 @@ enum class WithinCollection : uint8_t; template<> const char* EnumUtil::ToChars(AccessMode value); +template<> +const char* EnumUtil::ToChars(AggregateCombineType value); + template<> const char* EnumUtil::ToChars(AggregateHandling value); @@ -403,6 +414,9 @@ const char* EnumUtil::ToChars(DefaultOrderByNullType val template<> const char* EnumUtil::ToChars(DependencyEntryType value); +template<> +const char* EnumUtil::ToChars(DeprecatedIndexType value); + template<> const char* EnumUtil::ToChars(DistinctType value); @@ -427,6 +441,9 @@ const char* EnumUtil::ToChars(ExpressionType value); template<> const char* EnumUtil::ToChars(ExtensionLoadResult value); +template<> +const char* EnumUtil::ToChars(ExtraDropInfoType value); + template<> const char* EnumUtil::ToChars(ExtraTypeInfoType value); @@ -452,7 +469,7 @@ template<> const char* EnumUtil::ToChars(FunctionNullHandling value); template<> -const char* EnumUtil::ToChars(FunctionSideEffects value); +const char* EnumUtil::ToChars(FunctionStability value); template<> const char* EnumUtil::ToChars(HLLStorageType value); @@ -460,9 +477,6 @@ const char* EnumUtil::ToChars(HLLStorageType value); template<> const char* EnumUtil::ToChars(IndexConstraintType value); -template<> -const char* EnumUtil::ToChars(IndexType value); - template<> const char* EnumUtil::ToChars(InsertColumnOrder value); @@ -538,9 +552,6 @@ const char* EnumUtil::ToChars(ParseInfoType value); template<> const char* EnumUtil::ToChars(ParserExtensionResultType value); -template<> -const char* EnumUtil::ToChars(ParserMode value); - template<> const char* EnumUtil::ToChars(PartitionSortStage value); @@ -592,6 +603,12 @@ const char* EnumUtil::ToChars(ResultModifierType value); template<> const char* EnumUtil::ToChars(SampleMethod value); +template<> +const char* EnumUtil::ToChars(SecretDisplayType value); + +template<> +const char* EnumUtil::ToChars(SecretPersistType value); + template<> const char* EnumUtil::ToChars(SequenceInfo value); @@ -604,6 +621,9 @@ const char* EnumUtil::ToChars(SetScope value); template<> const char* EnumUtil::ToChars(SetType value); +template<> +const char* EnumUtil::ToChars(ShowType value); + template<> const char* EnumUtil::ToChars(SimplifiedTokenType value); @@ -707,6 +727,9 @@ const char* EnumUtil::ToChars(WithinCollection value); template<> AccessMode EnumUtil::FromString(const char *value); +template<> +AggregateCombineType EnumUtil::FromString(const char *value); + template<> AggregateHandling EnumUtil::FromString(const char *value); @@ -806,6 +829,9 @@ DefaultOrderByNullType EnumUtil::FromString(const char * template<> DependencyEntryType EnumUtil::FromString(const char *value); +template<> +DeprecatedIndexType EnumUtil::FromString(const char *value); + template<> DistinctType EnumUtil::FromString(const char *value); @@ -830,6 +856,9 @@ ExpressionType EnumUtil::FromString(const char *value); template<> ExtensionLoadResult EnumUtil::FromString(const char *value); +template<> +ExtraDropInfoType EnumUtil::FromString(const char *value); + template<> ExtraTypeInfoType EnumUtil::FromString(const char *value); @@ -855,7 +884,7 @@ template<> FunctionNullHandling EnumUtil::FromString(const char *value); template<> -FunctionSideEffects EnumUtil::FromString(const char *value); +FunctionStability EnumUtil::FromString(const char *value); template<> HLLStorageType EnumUtil::FromString(const char *value); @@ -863,9 +892,6 @@ HLLStorageType EnumUtil::FromString(const char *value); template<> IndexConstraintType EnumUtil::FromString(const char *value); -template<> -IndexType EnumUtil::FromString(const char *value); - template<> InsertColumnOrder EnumUtil::FromString(const char *value); @@ -941,9 +967,6 @@ ParseInfoType EnumUtil::FromString(const char *value); template<> ParserExtensionResultType EnumUtil::FromString(const char *value); -template<> -ParserMode EnumUtil::FromString(const char *value); - template<> PartitionSortStage EnumUtil::FromString(const char *value); @@ -995,6 +1018,12 @@ ResultModifierType EnumUtil::FromString(const char *value); template<> SampleMethod EnumUtil::FromString(const char *value); +template<> +SecretDisplayType EnumUtil::FromString(const char *value); + +template<> +SecretPersistType EnumUtil::FromString(const char *value); + template<> SequenceInfo EnumUtil::FromString(const char *value); @@ -1007,6 +1036,9 @@ SetScope EnumUtil::FromString(const char *value); template<> SetType EnumUtil::FromString(const char *value); +template<> +ShowType EnumUtil::FromString(const char *value); + template<> SimplifiedTokenType EnumUtil::FromString(const char *value); diff --git a/src/duckdb/src/include/duckdb/common/enums/catalog_type.hpp b/src/duckdb/src/include/duckdb/common/enums/catalog_type.hpp index b4c441005..3e18e666e 100644 --- a/src/duckdb/src/include/duckdb/common/enums/catalog_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/catalog_type.hpp @@ -40,8 +40,14 @@ enum class CatalogType : uint8_t { DELETED_ENTRY = 51, RENAMED_ENTRY = 52, + // secrets + SECRET_ENTRY = 71, + SECRET_TYPE_ENTRY = 72, + SECRET_FUNCTION_ENTRY = 73, + // dependency info DEPENDENCY_ENTRY = 100 + }; DUCKDB_API string CatalogTypeToString(CatalogType type); diff --git a/src/duckdb/src/include/duckdb/common/enums/compression_type.hpp b/src/duckdb/src/include/duckdb/common/enums/compression_type.hpp index e884d52a5..19bfa4cc8 100644 --- a/src/duckdb/src/include/duckdb/common/enums/compression_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/compression_type.hpp @@ -24,9 +24,12 @@ enum class CompressionType : uint8_t { COMPRESSION_FSST = 7, COMPRESSION_CHIMP = 8, COMPRESSION_PATAS = 9, + COMPRESSION_ALP = 10, + COMPRESSION_ALPRD = 11, COMPRESSION_COUNT // This has to stay the last entry of the type! }; +bool CompressionTypeIsDeprecated(CompressionType compression_type); vector ListCompressionTypes(void); CompressionType CompressionTypeFromString(const string &str); string CompressionTypeToString(CompressionType type); diff --git a/src/duckdb/src/include/duckdb/common/enums/index_constraint_type.hpp b/src/duckdb/src/include/duckdb/common/enums/index_constraint_type.hpp index 7777a2967..1e40dd4f3 100644 --- a/src/duckdb/src/include/duckdb/common/enums/index_constraint_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/index_constraint_type.hpp @@ -26,7 +26,7 @@ enum class IndexConstraintType : uint8_t { // Index Types //===--------------------------------------------------------------------===// // NOTE: deprecated. Still necessary to read older duckdb files. -enum class IndexType : uint8_t { +enum class DeprecatedIndexType : uint8_t { INVALID = 0, // invalid index type ART = 1, // Adaptive Radix Tree EXTENSION = 100 // Extension index diff --git a/src/duckdb/src/include/duckdb/common/enums/join_type.hpp b/src/duckdb/src/include/duckdb/common/enums/join_type.hpp index 4c9529768..4b3446963 100644 --- a/src/duckdb/src/include/duckdb/common/enums/join_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/join_type.hpp @@ -39,9 +39,15 @@ bool IsLeftOuterJoin(JoinType type); //! True if join is rght or full outer join bool IsRightOuterJoin(JoinType type); -//! +//! Whether the build side is propagated out of the join bool PropagatesBuildSide(JoinType type); +//! Whether the JoinType has an inverse +bool HasInverseJoinType(JoinType type); + +//! Gets the inverse JoinType, e.g., LEFT -> RIGHT +JoinType InverseJoinType(JoinType type); + // **DEPRECATED**: Use EnumUtil directly instead. string JoinTypeToString(JoinType type); diff --git a/src/duckdb/src/include/duckdb/common/enums/logical_operator_type.hpp b/src/duckdb/src/include/duckdb/common/enums/logical_operator_type.hpp index 02129b466..9c90ab7d2 100644 --- a/src/duckdb/src/include/duckdb/common/enums/logical_operator_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/logical_operator_type.hpp @@ -91,11 +91,6 @@ enum class LogicalOperatorType : uint8_t { // ----------------------------- LOGICAL_EXPLAIN = 150, - // ----------------------------- - // Show - // ----------------------------- - LOGICAL_SHOW = 160, - // ----------------------------- // Helpers // ----------------------------- @@ -107,6 +102,11 @@ enum class LogicalOperatorType : uint8_t { LOGICAL_LOAD = 180, LOGICAL_RESET = 181, + // ----------------------------- + // Secrets + // ----------------------------- + LOGICAL_CREATE_SECRET = 190, + LOGICAL_EXTENSION_OPERATOR = 255 }; diff --git a/src/duckdb/src/include/duckdb/common/enums/physical_operator_type.hpp b/src/duckdb/src/include/duckdb/common/enums/physical_operator_type.hpp index d83587ffd..3505953a4 100644 --- a/src/duckdb/src/include/duckdb/common/enums/physical_operator_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/physical_operator_type.hpp @@ -59,7 +59,8 @@ enum class PhysicalOperatorType : uint8_t { CROSS_PRODUCT, PIECEWISE_MERGE_JOIN, IE_JOIN, - DELIM_JOIN, + LEFT_DELIM_JOIN, + RIGHT_DELIM_JOIN, POSITIONAL_JOIN, ASOF_JOIN, // ----------------------------- @@ -111,7 +112,12 @@ enum class PhysicalOperatorType : uint8_t { INOUT_FUNCTION, RESULT_COLLECTOR, RESET, - EXTENSION + EXTENSION, + + // ----------------------------- + // Secret + // ----------------------------- + CREATE_SECRET, }; string PhysicalOperatorToString(PhysicalOperatorType type); diff --git a/src/duckdb/src/include/duckdb/common/enums/statement_type.hpp b/src/duckdb/src/include/duckdb/common/enums/statement_type.hpp index 61fb0c360..3a52b5283 100644 --- a/src/duckdb/src/include/duckdb/common/enums/statement_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/statement_type.hpp @@ -35,7 +35,6 @@ enum class StatementType : uint8_t { DROP_STATEMENT, // DROP statement type EXPORT_STATEMENT, // EXPORT statement type PRAGMA_STATEMENT, // PRAGMA statement type - SHOW_STATEMENT, // SHOW statement type VACUUM_STATEMENT, // VACUUM statement type CALL_STATEMENT, // CALL statement type SET_STATEMENT, // SET statement type @@ -46,8 +45,7 @@ enum class StatementType : uint8_t { ATTACH_STATEMENT, DETACH_STATEMENT, MULTI_STATEMENT, - COPY_DATABASE_STATEMENT - + COPY_DATABASE_STATEMENT, }; DUCKDB_API string StatementTypeToString(StatementType type); @@ -64,13 +62,13 @@ string StatementReturnTypeToString(StatementReturnType type); struct StatementProperties { StatementProperties() : requires_valid_transaction(true), allow_stream_result(false), bound_all_parameters(true), - return_type(StatementReturnType::QUERY_RESULT), parameter_count(0) { + return_type(StatementReturnType::QUERY_RESULT), parameter_count(0), always_require_rebind(false) { } //! The set of databases this statement will modify unordered_set modified_databases; //! Whether or not the statement requires a valid transaction. Almost all statements require this, with the - //! exception of + //! exception of ROLLBACK bool requires_valid_transaction; //! Whether or not the result can be streamed to the client bool allow_stream_result; @@ -80,6 +78,8 @@ struct StatementProperties { StatementReturnType return_type; //! The number of prepared statement parameters idx_t parameter_count; + //! Whether or not the statement ALWAYS requires a rebind + bool always_require_rebind; bool IsReadOnly() { return modified_databases.empty(); diff --git a/src/duckdb/src/include/duckdb/common/enums/tableref_type.hpp b/src/duckdb/src/include/duckdb/common/enums/tableref_type.hpp index 1f042c160..c4cf9844a 100644 --- a/src/duckdb/src/include/duckdb/common/enums/tableref_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/tableref_type.hpp @@ -23,8 +23,9 @@ enum class TableReferenceType : uint8_t { TABLE_FUNCTION = 5, // table producing function EXPRESSION_LIST = 6, // expression list CTE = 7, // Recursive CTE - EMPTY = 8, // placeholder for empty FROM - PIVOT = 9 // pivot statement + EMPTY_FROM = 8, // placeholder for empty FROM + PIVOT = 9, // pivot statement + SHOW_REF = 10 // SHOW statement }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/enums/wal_type.hpp b/src/duckdb/src/include/duckdb/common/enums/wal_type.hpp index b18fa9f16..4972be1b8 100644 --- a/src/duckdb/src/include/duckdb/common/enums/wal_type.hpp +++ b/src/duckdb/src/include/duckdb/common/enums/wal_type.hpp @@ -54,6 +54,7 @@ enum class WALType : uint8_t { // ----------------------------- // Flush // ----------------------------- + WAL_VERSION = 98, CHECKPOINT = 99, WAL_FLUSH = 100 }; diff --git a/src/duckdb/src/include/duckdb/common/preserved_error.hpp b/src/duckdb/src/include/duckdb/common/error_data.hpp similarity index 50% rename from src/duckdb/src/include/duckdb/common/preserved_error.hpp rename to src/duckdb/src/include/duckdb/common/error_data.hpp index 65bc4faec..83d636d32 100644 --- a/src/duckdb/src/include/duckdb/common/preserved_error.hpp +++ b/src/duckdb/src/include/duckdb/common/error_data.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/common/preserved_error.hpp +// duckdb/common/error_data.hpp // // //===----------------------------------------------------------------------===// @@ -12,37 +12,49 @@ #include "duckdb/common/string.hpp" namespace duckdb { +class ParsedExpression; +class TableRef; -class PreservedError { +class ErrorData { public: //! Not initialized, default constructor - DUCKDB_API PreservedError(); + DUCKDB_API ErrorData(); //! From std::exception - PreservedError(const std::exception &ex) : PreservedError(ex.what()) { - } + DUCKDB_API ErrorData(const std::exception &ex); // NOLINT: allow implicit construction from exception + //! From a raw string and exception type + DUCKDB_API explicit ErrorData(ExceptionType type, const string &raw_message); //! From a raw string - DUCKDB_API explicit PreservedError(const string &raw_message); - //! From an Exception - DUCKDB_API PreservedError(const Exception &exception); + DUCKDB_API explicit ErrorData(const string &raw_message); public: //! Throw the error [[noreturn]] DUCKDB_API void Throw(const string &prepended_message = "") const; //! Get the internal exception type of the error DUCKDB_API const ExceptionType &Type() const; - //! Allows adding addition information to the message - DUCKDB_API PreservedError &AddToMessage(const string &prepended_message); //! Used in clients like C-API, creates the final message and returns a reference to it DUCKDB_API const string &Message(); - //! Let's us do things like 'if (error)' - DUCKDB_API operator bool() const; - DUCKDB_API bool operator==(const PreservedError &other) const; - const shared_ptr &GetError() { - return exception_instance; + DUCKDB_API const string &RawMessage() { + return raw_message; + } + DUCKDB_API bool operator==(const ErrorData &other) const; + + inline bool HasError() const { + return initialized; } + const unordered_map &ExtraInfo() const { + return extra_info; + } + + DUCKDB_API void AddErrorLocation(const string &query); + DUCKDB_API void ConvertErrorToJSON(); + + DUCKDB_API void AddQueryLocation(optional_idx query_location); + DUCKDB_API void AddQueryLocation(QueryErrorContext error_context); + DUCKDB_API void AddQueryLocation(const ParsedExpression &ref); + DUCKDB_API void AddQueryLocation(const TableRef &ref); private: - //! Whether this PreservedError contains an exception or not + //! Whether this ErrorData contains an exception or not bool initialized; //! The ExceptionType of the preserved exception ExceptionType type; @@ -50,7 +62,8 @@ class PreservedError { string raw_message; //! The final message (stored in the preserved error for compatibility reasons with C-API) string final_message; - std::shared_ptr exception_instance; + //! Extra exception info + unordered_map extra_info; private: DUCKDB_API static string SanitizeErrorMessage(string error); diff --git a/src/duckdb/src/include/duckdb/common/exception.hpp b/src/duckdb/src/include/duckdb/common/exception.hpp index 5e2ad4d6e..bd5f1857a 100644 --- a/src/duckdb/src/include/duckdb/common/exception.hpp +++ b/src/duckdb/src/include/duckdb/common/exception.hpp @@ -11,7 +11,7 @@ #include "duckdb/common/assert.hpp" #include "duckdb/common/exception_format_value.hpp" #include "duckdb/common/shared_ptr.hpp" -#include "duckdb/common/map.hpp" +#include "duckdb/common/unordered_map.hpp" #include "duckdb/common/typedefs.hpp" #include @@ -20,7 +20,12 @@ namespace duckdb { enum class PhysicalType : uint8_t; struct LogicalType; +class Expression; +class ParsedExpression; +class QueryErrorContext; +class TableRef; struct hugeint_t; +class optional_idx; inline void assert_restrict_function(const void *left_start, const void *left_end, const void *right_start, const void *right_end, const char *fname, int linenr) { @@ -81,29 +86,19 @@ enum class ExceptionType { DEPENDENCY = 37, // dependency HTTP = 38, MISSING_EXTENSION = 39, // Thrown when an extension is used but not loaded - AUTOLOAD = 40 // Thrown when an extension is used but not loaded + AUTOLOAD = 40, // Thrown when an extension is used but not loaded + SEQUENCE = 41 }; -class HTTPException; -class Exception : public std::exception { +class Exception : public std::runtime_error { public: - DUCKDB_API explicit Exception(const string &msg); DUCKDB_API Exception(ExceptionType exception_type, const string &message); - - ExceptionType type; + DUCKDB_API Exception(ExceptionType exception_type, const string &message, + const unordered_map &extra_info); public: - DUCKDB_API const char *what() const noexcept override; - DUCKDB_API const string &RawMessage() const; - DUCKDB_API static string ExceptionTypeToString(ExceptionType type); DUCKDB_API static ExceptionType StringToExceptionType(const string &type); - [[noreturn]] DUCKDB_API static void ThrowAsTypeWithMessage(ExceptionType type, const string &message, - const std::shared_ptr &original); - virtual std::shared_ptr Copy() const { - return make_shared(type, raw_message_); - } - DUCKDB_API const HTTPException &AsHTTPException() const; template static string ConstructMessage(const string &msg, Args... params) { @@ -114,6 +109,21 @@ class Exception : public std::exception { return ConstructMessageRecursive(msg, values, params...); } + DUCKDB_API static unordered_map InitializeExtraInfo(const Expression &expr); + DUCKDB_API static unordered_map InitializeExtraInfo(const ParsedExpression &expr); + DUCKDB_API static unordered_map InitializeExtraInfo(const QueryErrorContext &error_context); + DUCKDB_API static unordered_map InitializeExtraInfo(const TableRef &ref); + DUCKDB_API static unordered_map InitializeExtraInfo(optional_idx error_location); + DUCKDB_API static unordered_map InitializeExtraInfo(const string &subtype, + optional_idx error_location); + + DUCKDB_API static string ToJSON(ExceptionType type, const string &message); + DUCKDB_API static string ToJSON(ExceptionType type, const string &message, + const unordered_map &extra_info); + + DUCKDB_API static bool InvalidatesTransaction(ExceptionType exception_type); + DUCKDB_API static bool InvalidatesDatabase(ExceptionType exception_type); + DUCKDB_API static string ConstructMessageRecursive(const string &msg, std::vector &values); template @@ -130,31 +140,13 @@ class Exception : public std::exception { return (message + "\n" + GetStackTrace()); } -private: - string exception_message_; - string raw_message_; + DUCKDB_API static void SetQueryLocation(optional_idx error_location, unordered_map &extra_info); }; //===--------------------------------------------------------------------===// // Exception derived classes //===--------------------------------------------------------------------===// - -//! Exceptions that are StandardExceptions do NOT invalidate the current transaction when thrown -class StandardException : public Exception { -public: - DUCKDB_API StandardException(ExceptionType exception_type, const string &message); -}; - -class CatalogException : public StandardException { -public: - DUCKDB_API explicit CatalogException(const string &msg); - - template - explicit CatalogException(const string &msg, Args... params) : CatalogException(ConstructMessage(msg, params...)) { - } -}; - -class ConnectionException : public StandardException { +class ConnectionException : public Exception { public: DUCKDB_API explicit ConnectionException(const string &msg); @@ -164,16 +156,7 @@ class ConnectionException : public StandardException { } }; -class ParserException : public StandardException { -public: - DUCKDB_API explicit ParserException(const string &msg); - - template - explicit ParserException(const string &msg, Args... params) : ParserException(ConstructMessage(msg, params...)) { - } -}; - -class PermissionException : public StandardException { +class PermissionException : public Exception { public: DUCKDB_API explicit PermissionException(const string &msg); @@ -183,45 +166,6 @@ class PermissionException : public StandardException { } }; -class BinderException : public StandardException { -public: - DUCKDB_API explicit BinderException(const string &msg); - - template - explicit BinderException(const string &msg, Args... params) : BinderException(ConstructMessage(msg, params...)) { - } -}; - -class ConversionException : public Exception { -public: - DUCKDB_API explicit ConversionException(const string &msg); - - template - explicit ConversionException(const string &msg, Args... params) - : ConversionException(ConstructMessage(msg, params...)) { - } -}; - -class TransactionException : public Exception { -public: - DUCKDB_API explicit TransactionException(const string &msg); - - template - explicit TransactionException(const string &msg, Args... params) - : TransactionException(ConstructMessage(msg, params...)) { - } -}; - -class NotImplementedException : public Exception { -public: - DUCKDB_API explicit NotImplementedException(const string &msg); - - template - explicit NotImplementedException(const string &msg, Args... params) - : NotImplementedException(ConstructMessage(msg, params...)) { - } -}; - class OutOfRangeException : public Exception { public: DUCKDB_API explicit OutOfRangeException(const string &msg); @@ -230,6 +174,10 @@ class OutOfRangeException : public Exception { explicit OutOfRangeException(const string &msg, Args... params) : OutOfRangeException(ConstructMessage(msg, params...)) { } + DUCKDB_API OutOfRangeException(const int64_t value, const PhysicalType origType, const PhysicalType newType); + DUCKDB_API OutOfRangeException(const hugeint_t value, const PhysicalType origType, const PhysicalType newType); + DUCKDB_API OutOfRangeException(const double value, const PhysicalType origType, const PhysicalType newType); + DUCKDB_API OutOfRangeException(const PhysicalType varType, const idx_t length); }; class OutOfMemoryException : public Exception { @@ -292,74 +240,19 @@ class MissingExtensionException : public Exception { } }; -class AutoloadException : public Exception { +class NotImplementedException : public Exception { public: - DUCKDB_API explicit AutoloadException(const string &extension_name, Exception &e); + DUCKDB_API explicit NotImplementedException(const string &msg); template - explicit AutoloadException(const string &extension_name, Exception &e, Args... params) - : AutoloadException(ConstructMessage(extension_name, e, params...)) { + explicit NotImplementedException(const string &msg, Args... params) + : NotImplementedException(ConstructMessage(msg, params...)) { } - -protected: - Exception &wrapped_exception; }; -class HTTPException : public IOException { +class AutoloadException : public Exception { public: - template - struct ResponseShape { - typedef int status; - }; - - template ::status = 0, typename... ARGS> - explicit HTTPException(RESPONSE &response, const string &msg, ARGS... params) - : HTTPException(response.status, response.body, response.headers, response.reason, msg, params...) { - } - - template - struct ResponseWrapperShape { - typedef int code; - }; - template ::code = 0, typename... ARGS> - explicit HTTPException(RESPONSE &response, const string &msg, ARGS... params) - : HTTPException(response.code, response.body, response.headers, response.error, msg, params...) { - } - - template - explicit HTTPException(int status_code, string response_body, HEADERS headers, const string &reason, - const string &msg, ARGS... params) - : IOException(ExceptionType::HTTP, ConstructMessage(msg, params...)), status_code(status_code), reason(reason), - response_body(std::move(response_body)) { - this->headers.insert(headers.begin(), headers.end()); - D_ASSERT(this->headers.size() > 0); - } - - std::shared_ptr Copy() const { - return make_shared(status_code, response_body, headers, reason, RawMessage()); - } - - const std::multimap GetHeaders() const { - return headers; - } - int GetStatusCode() const { - return status_code; - } - const string &GetResponseBody() const { - return response_body; - } - const string &GetReason() const { - return reason; - } - [[noreturn]] void Throw() const { - throw HTTPException(status_code, response_body, headers, reason, RawMessage()); - } - -private: - int status_code; - string reason; - string response_body; - std::multimap headers; + DUCKDB_API explicit AutoloadException(const string &extension_name, const string &message); }; class SerializationException : public Exception { @@ -403,7 +296,7 @@ class FatalException : public Exception { } }; -class InternalException : public FatalException { +class InternalException : public Exception { public: DUCKDB_API explicit InternalException(const string &msg); @@ -416,19 +309,16 @@ class InternalException : public FatalException { class InvalidInputException : public Exception { public: DUCKDB_API explicit InvalidInputException(const string &msg); + DUCKDB_API explicit InvalidInputException(const string &msg, const unordered_map &extra_info); template explicit InvalidInputException(const string &msg, Args... params) : InvalidInputException(ConstructMessage(msg, params...)) { } -}; - -class CastException : public Exception { -public: - DUCKDB_API CastException(const PhysicalType origType, const PhysicalType newType); - DUCKDB_API CastException(const LogicalType &origType, const LogicalType &newType); - DUCKDB_API - CastException(const string &msg); //! Needed to be able to recreate the exception after it's been serialized + template + explicit InvalidInputException(Expression &expr, const string &msg, Args... params) + : InvalidInputException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(expr)) { + } }; class InvalidTypeException : public Exception { @@ -447,17 +337,7 @@ class TypeMismatchException : public Exception { TypeMismatchException(const string &msg); //! Needed to be able to recreate the exception after it's been serialized }; -class ValueOutOfRangeException : public Exception { -public: - DUCKDB_API ValueOutOfRangeException(const int64_t value, const PhysicalType origType, const PhysicalType newType); - DUCKDB_API ValueOutOfRangeException(const hugeint_t value, const PhysicalType origType, const PhysicalType newType); - DUCKDB_API ValueOutOfRangeException(const double value, const PhysicalType origType, const PhysicalType newType); - DUCKDB_API ValueOutOfRangeException(const PhysicalType varType, const idx_t length); - DUCKDB_API ValueOutOfRangeException( - const string &msg); //! Needed to be able to recreate the exception after it's been serialized -}; - -class ParameterNotAllowedException : public StandardException { +class ParameterNotAllowedException : public Exception { public: DUCKDB_API explicit ParameterNotAllowedException(const string &msg); diff --git a/src/duckdb/src/include/duckdb/common/exception/binder_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/binder_exception.hpp new file mode 100644 index 000000000..493e3e2f1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/binder_exception.hpp @@ -0,0 +1,47 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/binder_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" +#include "duckdb/parser/query_error_context.hpp" + +namespace duckdb { + +class BinderException : public Exception { +public: + DUCKDB_API explicit BinderException(const string &msg, const unordered_map &extra_info); + DUCKDB_API explicit BinderException(const string &msg); + + template + explicit BinderException(const string &msg, Args... params) : BinderException(ConstructMessage(msg, params...)) { + } + template + explicit BinderException(const TableRef &ref, const string &msg, Args... params) + : BinderException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(ref)) { + } + template + explicit BinderException(const ParsedExpression &expr, const string &msg, Args... params) + : BinderException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(expr)) { + } + template + explicit BinderException(QueryErrorContext error_context, const string &msg, Args... params) + : BinderException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(error_context)) { + } + template + explicit BinderException(optional_idx error_location, const string &msg, Args... params) + : BinderException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(error_location)) { + } + + static BinderException ColumnNotFound(const string &name, const vector &similar_bindings, + QueryErrorContext context = QueryErrorContext()); + static BinderException NoMatchingFunction(const string &name, const vector &arguments, + const vector &candidates); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception/catalog_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/catalog_exception.hpp new file mode 100644 index 000000000..1c9932055 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/catalog_exception.hpp @@ -0,0 +1,39 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/catalog_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" +#include "duckdb/common/enums/catalog_type.hpp" +#include "duckdb/parser/query_error_context.hpp" +#include "duckdb/common/unordered_map.hpp" + +namespace duckdb { + +class CatalogException : public Exception { +public: + DUCKDB_API explicit CatalogException(const string &msg); + DUCKDB_API explicit CatalogException(const string &msg, const unordered_map &extra_info); + + template + explicit CatalogException(const string &msg, Args... params) : CatalogException(ConstructMessage(msg, params...)) { + } + template + explicit CatalogException(QueryErrorContext error_context, const string &msg, Args... params) + : CatalogException(ConstructMessage(msg, params...), Exception::InitializeExtraInfo(error_context)) { + } + + static CatalogException MissingEntry(CatalogType type, const string &name, const string &suggestion, + QueryErrorContext context = QueryErrorContext()); + static CatalogException MissingEntry(const string &type, const string &name, const vector &suggestions, + QueryErrorContext context = QueryErrorContext()); + static CatalogException EntryAlreadyExists(CatalogType type, const string &name, + QueryErrorContext context = QueryErrorContext()); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception/conversion_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/conversion_exception.hpp new file mode 100644 index 000000000..945e091d6 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/conversion_exception.hpp @@ -0,0 +1,27 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/conversion_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" + +namespace duckdb { + +class ConversionException : public Exception { +public: + DUCKDB_API explicit ConversionException(const string &msg); + DUCKDB_API ConversionException(const PhysicalType origType, const PhysicalType newType); + DUCKDB_API ConversionException(const LogicalType &origType, const LogicalType &newType); + + template + explicit ConversionException(const string &msg, Args... params) + : ConversionException(ConstructMessage(msg, params...)) { + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception/http_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/http_exception.hpp new file mode 100644 index 000000000..c953ee3b1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/http_exception.hpp @@ -0,0 +1,62 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/http_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" +#include "duckdb/common/to_string.hpp" + +namespace duckdb { + +class HTTPException : public Exception { +public: + template + struct ResponseShape { + typedef int status; + }; + + explicit HTTPException(string message) : Exception(ExceptionType::HTTP, std::move(message)) { + } + + template ::status = 0, typename... ARGS> + explicit HTTPException(RESPONSE &response, const string &msg, ARGS... params) + : HTTPException(response.status, response.body, response.headers, response.reason, msg, params...) { + } + + template + struct ResponseWrapperShape { + typedef int code; + }; + + template ::code = 0, typename... ARGS> + explicit HTTPException(RESPONSE &response, const string &msg, ARGS... params) + : HTTPException(response.code, response.body, response.headers, response.error, msg, params...) { + } + + template + explicit HTTPException(int status_code, const string &response_body, const HEADERS &headers, const string &reason, + const string &msg, ARGS... params) + : Exception(ExceptionType::HTTP, ConstructMessage(msg, params...), + HTTPExtraInfo(status_code, response_body, headers, reason)) { + } + + template + static unordered_map HTTPExtraInfo(int status_code, const string &response_body, + const HEADERS &headers, const string &reason) { + unordered_map extra_info; + extra_info["status_code"] = to_string(status_code); + extra_info["reason"] = reason; + extra_info["response_body"] = response_body; + for (auto &entry : headers) { + extra_info["header_" + entry.first] = entry.second; + } + return extra_info; + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception/list.hpp b/src/duckdb/src/include/duckdb/common/exception/list.hpp new file mode 100644 index 000000000..532b77d7b --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/list.hpp @@ -0,0 +1,6 @@ +#include "duckdb/common/exception/binder_exception.hpp" +#include "duckdb/common/exception/catalog_exception.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" +#include "duckdb/common/exception/http_exception.hpp" +#include "duckdb/common/exception/parser_exception.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" diff --git a/src/duckdb/src/include/duckdb/common/exception/parser_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/parser_exception.hpp new file mode 100644 index 000000000..15d97bf0e --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/parser_exception.hpp @@ -0,0 +1,29 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/parser_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" +#include "duckdb/common/optional_idx.hpp" +#include "duckdb/common/unordered_map.hpp" + +namespace duckdb { + +class ParserException : public Exception { +public: + DUCKDB_API explicit ParserException(const string &msg); + DUCKDB_API explicit ParserException(const string &msg, const unordered_map &extra_info); + + template + explicit ParserException(const string &msg, Args... params) : ParserException(ConstructMessage(msg, params...)) { + } + + static ParserException SyntaxError(const string &query, const string &error_message, optional_idx error_location); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception/transaction_exception.hpp b/src/duckdb/src/include/duckdb/common/exception/transaction_exception.hpp new file mode 100644 index 000000000..2dadb8d09 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/exception/transaction_exception.hpp @@ -0,0 +1,25 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/exception/transaction_exception.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/exception.hpp" + +namespace duckdb { + +class TransactionException : public Exception { +public: + DUCKDB_API explicit TransactionException(const string &msg); + + template + explicit TransactionException(const string &msg, Args... params) + : TransactionException(ConstructMessage(msg, params...)) { + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/exception_format_value.hpp b/src/duckdb/src/include/duckdb/common/exception_format_value.hpp index 1834663e5..286a2b28a 100644 --- a/src/duckdb/src/include/duckdb/common/exception_format_value.hpp +++ b/src/duckdb/src/include/duckdb/common/exception_format_value.hpp @@ -47,10 +47,11 @@ enum class ExceptionFormatValueType : uint8_t { }; struct ExceptionFormatValue { - DUCKDB_API ExceptionFormatValue(double dbl_val); // NOLINT - DUCKDB_API ExceptionFormatValue(int64_t int_val); // NOLINT - DUCKDB_API ExceptionFormatValue(string str_val); // NOLINT - DUCKDB_API ExceptionFormatValue(hugeint_t hg_val); // NOLINT + DUCKDB_API ExceptionFormatValue(double dbl_val); // NOLINT + DUCKDB_API ExceptionFormatValue(int64_t int_val); // NOLINT + DUCKDB_API ExceptionFormatValue(string str_val); // NOLINT + DUCKDB_API ExceptionFormatValue(hugeint_t hg_val); // NOLINT + DUCKDB_API ExceptionFormatValue(uhugeint_t uhg_val); // NOLINT ExceptionFormatValueType type; @@ -86,5 +87,7 @@ template <> DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(char *value); template <> DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(hugeint_t value); +template <> +DUCKDB_API ExceptionFormatValue ExceptionFormatValue::CreateFormatValue(uhugeint_t value); } // 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 32ccdd32c..930c44a33 100644 --- a/src/duckdb/src/include/duckdb/common/extra_type_info.hpp +++ b/src/duckdb/src/include/duckdb/common/extra_type_info.hpp @@ -25,6 +25,8 @@ enum class ExtraTypeInfoType : uint8_t { USER_TYPE_INFO = 7, AGGREGATE_STATE_TYPE_INFO = 8, ARRAY_TYPE_INFO = 9, + ANY_TYPE_INFO = 10, + INTEGER_LITERAL_TYPE_INFO = 11 }; struct ExtraTypeInfo { @@ -199,4 +201,37 @@ struct ArrayTypeInfo : public ExtraTypeInfo { bool EqualsInternal(ExtraTypeInfo *other_p) const override; }; +struct AnyTypeInfo : public ExtraTypeInfo { + AnyTypeInfo(LogicalType target_type, idx_t cast_score); + + LogicalType target_type; + idx_t cast_score; + +public: + void Serialize(Serializer &serializer) const override; + static shared_ptr Deserialize(Deserializer &source); + +protected: + bool EqualsInternal(ExtraTypeInfo *other_p) const override; + +private: + AnyTypeInfo(); +}; + +struct IntegerLiteralTypeInfo : public ExtraTypeInfo { + IntegerLiteralTypeInfo(Value constant_value); + + Value constant_value; + +public: + void Serialize(Serializer &serializer) const override; + static shared_ptr Deserialize(Deserializer &source); + +protected: + bool EqualsInternal(ExtraTypeInfo *other_p) const override; + +private: + IntegerLiteralTypeInfo(); +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/helper.hpp b/src/duckdb/src/include/duckdb/common/helper.hpp index 944ad7a24..bfc994e5c 100644 --- a/src/duckdb/src/include/duckdb/common/helper.hpp +++ b/src/duckdb/src/include/duckdb/common/helper.hpp @@ -151,7 +151,7 @@ T MaxValue(T a, T b) { } template -T MinValue(T a, T b) { +constexpr T MinValue(T a, T b) { return a < b ? a : b; } @@ -160,12 +160,17 @@ T AbsValue(T a) { return a < 0 ? -a : a; } -//Align value (ceiling) +//! Align value (ceiling) template static inline T AlignValue(T n) { return ((n + (val - 1)) / val) * val; } +template +constexpr inline T AlignValueFloor(T n) { + return (n / val) * val; +} + template static inline bool ValueIsAligned(T n) { return (n % val) == 0; diff --git a/src/duckdb/src/include/duckdb/common/http_state.hpp b/src/duckdb/src/include/duckdb/common/http_state.hpp index ca60606b9..287d5a377 100644 --- a/src/duckdb/src/include/duckdb/common/http_state.hpp +++ b/src/duckdb/src/include/duckdb/common/http_state.hpp @@ -33,6 +33,8 @@ class CachedFile : public std::enable_shared_from_this { shared_ptr data; //! Data capacity uint64_t capacity = 0; + //! Size of file + idx_t size; //! Lock for initializing the file mutex lock; //! When initialized is set to true, the file is safe for parallel reading without holding the lock @@ -47,7 +49,7 @@ class CachedFileHandle { //! allocate a buffer for the file void AllocateBuffer(idx_t size); //! Indicate the file is fully downloaded and safe for parallel reading without lock - void SetInitialized(); + void SetInitialized(idx_t total_size); //! Grow buffer to new size, copying over `bytes_to_copy` to the new buffer void GrowBuffer(idx_t new_capacity, idx_t bytes_to_copy); //! Write to the buffer @@ -62,20 +64,26 @@ class CachedFileHandle { uint64_t GetCapacity() { return file->capacity; } + //! Return the size of the initialized file + idx_t GetSize() { + D_ASSERT(file->initialized); + return file->size; + } private: unique_ptr> lock; shared_ptr file; }; -class HTTPState { +class HTTPState : public ClientContextState { public: //! Reset all counters and cached files void Reset(); //! Get cache entry, create if not exists shared_ptr &GetCachedFile(const string &path); - //! Helper function to get the HTTP state - static shared_ptr TryGetState(FileOpener *opener); + //! Helper functions to get the HTTP state + static shared_ptr TryGetState(ClientContext &context, bool create_on_missing = true); + static shared_ptr TryGetState(FileOpener *opener, bool create_on_missing = true); bool IsEmpty() { return head_count == 0 && get_count == 0 && put_count == 0 && post_count == 0 && total_bytes_received == 0 && @@ -89,6 +97,11 @@ class HTTPState { atomic total_bytes_received {0}; atomic total_bytes_sent {0}; + //! Called by the ClientContext when the current query ends + void QueryEnd() override { + Reset(); + } + private: //! Mutex to lock when getting the cached file(Parallel Only) mutex cached_files_mutex; diff --git a/src/duckdb/src/include/duckdb/common/hugeint.hpp b/src/duckdb/src/include/duckdb/common/hugeint.hpp index e1bb454bf..fd58a8cc5 100644 --- a/src/duckdb/src/include/duckdb/common/hugeint.hpp +++ b/src/duckdb/src/include/duckdb/common/hugeint.hpp @@ -7,6 +7,9 @@ namespace duckdb { +// Forward declaration to allow conversion between hugeint and uhugeint +struct uhugeint_t; + struct hugeint_t { public: uint64_t lower; @@ -64,7 +67,7 @@ struct hugeint_t { DUCKDB_API explicit operator bool() const; DUCKDB_API bool operator!() const; - // cast operators + // cast operators -- doesn't check bounds/overflow/underflow DUCKDB_API explicit operator uint8_t() const; DUCKDB_API explicit operator uint16_t() const; DUCKDB_API explicit operator uint32_t() const; @@ -73,6 +76,7 @@ struct hugeint_t { DUCKDB_API explicit operator int16_t() const; DUCKDB_API explicit operator int32_t() const; DUCKDB_API explicit operator int64_t() const; + DUCKDB_API operator uhugeint_t() const; // NOLINT: Allow implicit conversion from `hugeint_t` }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/limits.hpp b/src/duckdb/src/include/duckdb/common/limits.hpp index 98b0afdd1..21e28f7d4 100644 --- a/src/duckdb/src/include/duckdb/common/limits.hpp +++ b/src/duckdb/src/include/duckdb/common/limits.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/hugeint.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/types.hpp" #include @@ -51,6 +52,23 @@ struct NumericLimits { } }; +template <> +struct NumericLimits { + static constexpr uhugeint_t Minimum() { + return {0, 0}; + }; + static constexpr uhugeint_t Maximum() { + return {std::numeric_limits::max(), std::numeric_limits::max()}; + }; + static constexpr bool IsSigned() { + return false; + } + + static constexpr idx_t Digits() { + return 39; + } +}; + template <> constexpr idx_t NumericLimits::Digits() { return 3; diff --git a/src/duckdb/src/include/duckdb/common/numeric_utils.hpp b/src/duckdb/src/include/duckdb/common/numeric_utils.hpp index 73ee61915..4ffcddc25 100644 --- a/src/duckdb/src/include/duckdb/common/numeric_utils.hpp +++ b/src/duckdb/src/include/duckdb/common/numeric_utils.hpp @@ -23,16 +23,24 @@ struct MakeSigned { using type = hugeint_t; }; +template <> +struct MakeSigned { + using type = hugeint_t; +}; + template struct MakeUnsigned { using type = typename std::make_unsigned::type; }; -// hugeint_t does not actually have an unsigned variant (yet), but this is required to make compression work -// if an unsigned variant gets implemented this (probably) can be changed without breaking anything template <> struct MakeUnsigned { - using type = hugeint_t; + using type = uhugeint_t; +}; + +template <> +struct MakeUnsigned { + using type = uhugeint_t; }; template @@ -45,4 +53,9 @@ struct IsIntegral { static constexpr bool value = true; }; +template <> +struct IsIntegral { + static constexpr bool value = true; +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/operator/add.hpp b/src/duckdb/src/include/duckdb/common/operator/add.hpp index b37e5ca82..ff9c3a060 100644 --- a/src/duckdb/src/include/duckdb/common/operator/add.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/add.hpp @@ -35,11 +35,15 @@ timestamp_t AddOperator::Operation(date_t left, dtime_t right); template <> timestamp_t AddOperator::Operation(dtime_t left, date_t right); template <> +timestamp_t AddOperator::Operation(date_t left, dtime_tz_t right); +template <> +timestamp_t AddOperator::Operation(dtime_tz_t left, date_t right); +template <> interval_t AddOperator::Operation(interval_t left, interval_t right); template <> -date_t AddOperator::Operation(date_t left, interval_t right); +timestamp_t AddOperator::Operation(date_t left, interval_t right); template <> -date_t AddOperator::Operation(interval_t left, date_t right); +timestamp_t AddOperator::Operation(interval_t left, date_t right); template <> timestamp_t AddOperator::Operation(timestamp_t left, interval_t right); template <> @@ -70,6 +74,8 @@ bool TryAddOperator::Operation(int32_t left, int32_t right, int32_t &result); template <> DUCKDB_API bool TryAddOperator::Operation(int64_t left, int64_t right, int64_t &result); template <> +bool TryAddOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result); +template <> bool TryAddOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t &result); struct AddOperatorOverflowCheck { @@ -126,4 +132,9 @@ dtime_t AddTimeOperator::Operation(dtime_t left, interval_t right); template <> dtime_t AddTimeOperator::Operation(interval_t left, dtime_t right); +template <> +dtime_tz_t AddTimeOperator::Operation(dtime_tz_t left, interval_t right); +template <> +dtime_tz_t AddTimeOperator::Operation(interval_t left, dtime_tz_t right); + } // namespace duckdb 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 a187defe3..0f9ae453b 100644 --- a/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/cast_operators.hpp @@ -20,6 +20,7 @@ #include "duckdb/common/types/null_value.hpp" #include "duckdb/common/types/bit.hpp" #include "duckdb/common/types/vector.hpp" +#include "duckdb/common/exception/conversion_exception.hpp" namespace duckdb { struct ValidityMask; @@ -99,6 +100,8 @@ DUCKDB_API bool TryCast::Operation(bool input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(bool input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(bool input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(bool input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(bool input, uint16_t &result, bool strict); @@ -127,6 +130,8 @@ DUCKDB_API bool TryCast::Operation(int8_t input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int8_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(int8_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(int8_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int8_t input, uint16_t &result, bool strict); @@ -155,6 +160,8 @@ DUCKDB_API bool TryCast::Operation(int16_t input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int16_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(int16_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(int16_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int16_t input, uint16_t &result, bool strict); @@ -183,6 +190,8 @@ DUCKDB_API bool TryCast::Operation(int32_t input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int32_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(int32_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(int32_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int32_t input, uint16_t &result, bool strict); @@ -211,6 +220,8 @@ DUCKDB_API bool TryCast::Operation(int64_t input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int64_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(int64_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(int64_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(int64_t input, uint16_t &result, bool strict); @@ -239,6 +250,8 @@ DUCKDB_API bool TryCast::Operation(hugeint_t input, int64_t &result, bool strict template <> DUCKDB_API bool TryCast::Operation(hugeint_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(hugeint_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(hugeint_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(hugeint_t input, uint16_t &result, bool strict); @@ -251,6 +264,36 @@ DUCKDB_API bool TryCast::Operation(hugeint_t input, float &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(hugeint_t input, double &result, bool strict); +//===--------------------------------------------------------------------===// +// Cast uhugeint_t -> Numeric +//===--------------------------------------------------------------------===// +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, bool &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, int8_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, int16_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, int32_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, int64_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, uhugeint_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, hugeint_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, uint8_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, uint16_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, uint32_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, uint64_t &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, float &result, bool strict); +template <> +DUCKDB_API bool TryCast::Operation(uhugeint_t input, double &result, bool strict); + //===--------------------------------------------------------------------===// // Cast uint8_t -> Numeric //===--------------------------------------------------------------------===// @@ -267,6 +310,8 @@ DUCKDB_API bool TryCast::Operation(uint8_t input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(uint8_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(uint8_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(uint8_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(uint8_t input, uint16_t &result, bool strict); @@ -295,6 +340,8 @@ DUCKDB_API bool TryCast::Operation(uint16_t input, int64_t &result, bool strict) template <> DUCKDB_API bool TryCast::Operation(uint16_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(uint16_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(uint16_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(uint16_t input, uint16_t &result, bool strict); @@ -323,6 +370,8 @@ DUCKDB_API bool TryCast::Operation(uint32_t input, int64_t &result, bool strict) template <> DUCKDB_API bool TryCast::Operation(uint32_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(uint32_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(uint32_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(uint32_t input, uint16_t &result, bool strict); @@ -351,6 +400,8 @@ DUCKDB_API bool TryCast::Operation(uint64_t input, int64_t &result, bool strict) template <> DUCKDB_API bool TryCast::Operation(uint64_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(uint64_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(uint64_t input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(uint64_t input, uint16_t &result, bool strict); @@ -379,6 +430,8 @@ DUCKDB_API bool TryCast::Operation(float input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(float input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(float input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(float input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(float input, uint16_t &result, bool strict); @@ -407,6 +460,8 @@ DUCKDB_API bool TryCast::Operation(double input, int64_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(double input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(double input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(double input, uint8_t &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(double input, uint16_t &result, bool strict); @@ -443,6 +498,8 @@ DUCKDB_API bool TryCast::Operation(string_t input, uint64_t &result, bool strict template <> DUCKDB_API bool TryCast::Operation(string_t input, hugeint_t &result, bool strict); template <> +DUCKDB_API bool TryCast::Operation(string_t input, uhugeint_t &result, bool strict); +template <> DUCKDB_API bool TryCast::Operation(string_t input, float &result, bool strict); template <> DUCKDB_API bool TryCast::Operation(string_t input, double &result, bool strict); @@ -805,6 +862,8 @@ template <> bool CastFromBitToNumeric::Operation(string_t input, bool &result, bool strict); template <> bool CastFromBitToNumeric::Operation(string_t input, hugeint_t &result, bool strict); +template <> +bool CastFromBitToNumeric::Operation(string_t input, uhugeint_t &result, bool strict); struct CastFromBitToBlob { template diff --git a/src/duckdb/src/include/duckdb/common/operator/convert_to_string.hpp b/src/duckdb/src/include/duckdb/common/operator/convert_to_string.hpp index c979265e2..89a54424f 100644 --- a/src/duckdb/src/include/duckdb/common/operator/convert_to_string.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/convert_to_string.hpp @@ -43,6 +43,8 @@ DUCKDB_API string ConvertToString::Operation(uint64_t input); template <> DUCKDB_API string ConvertToString::Operation(hugeint_t input); template <> +DUCKDB_API string ConvertToString::Operation(uhugeint_t input); +template <> DUCKDB_API string ConvertToString::Operation(float input); template <> DUCKDB_API string ConvertToString::Operation(double input); diff --git a/src/duckdb/src/include/duckdb/common/operator/decimal_cast_operators.hpp b/src/duckdb/src/include/duckdb/common/operator/decimal_cast_operators.hpp index 6610ccadb..847af0a22 100644 --- a/src/duckdb/src/include/duckdb/common/operator/decimal_cast_operators.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/decimal_cast_operators.hpp @@ -194,6 +194,35 @@ template <> bool TryCastFromDecimal::Operation(hugeint_t input, hugeint_t &result, string *error_message, uint8_t width, uint8_t scale); +//===--------------------------------------------------------------------===// +// Cast Decimal <-> uhugeint_t +//===--------------------------------------------------------------------===// +template <> +DUCKDB_API bool TryCastToDecimal::Operation(uhugeint_t input, int16_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +DUCKDB_API bool TryCastToDecimal::Operation(uhugeint_t input, int32_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +DUCKDB_API bool TryCastToDecimal::Operation(uhugeint_t input, int64_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +DUCKDB_API bool TryCastToDecimal::Operation(uhugeint_t input, hugeint_t &result, string *error_message, uint8_t width, + uint8_t scale); + +template <> +bool TryCastFromDecimal::Operation(int16_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +bool TryCastFromDecimal::Operation(int32_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +bool TryCastFromDecimal::Operation(int64_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale); +template <> +bool TryCastFromDecimal::Operation(hugeint_t input, uhugeint_t &result, string *error_message, uint8_t width, + uint8_t scale); + //===--------------------------------------------------------------------===// // Cast Decimal <-> uint8_t //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/include/duckdb/common/operator/double_cast_operator.hpp b/src/duckdb/src/include/duckdb/common/operator/double_cast_operator.hpp new file mode 100644 index 000000000..af87eaafa --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/operator/double_cast_operator.hpp @@ -0,0 +1,52 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/operator/double_cast_operator.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb.h" +#include "fast_float/fast_float.h" + +namespace duckdb { +template +static bool TryDoubleCast(const char *buf, idx_t len, T &result, bool strict, char decimal_separator = '.') { + // skip any spaces at the start + while (len > 0 && StringUtil::CharacterIsSpace(*buf)) { + buf++; + len--; + } + if (len == 0) { + return false; + } + if (*buf == '+') { + if (strict) { + // plus is not allowed in strict mode + return false; + } + buf++; + len--; + } + if (strict && len >= 2) { + if (buf[0] == '0' && StringUtil::CharacterIsDigit(buf[1])) { + // leading zeros are not allowed in strict mode + return false; + } + } + auto endptr = buf + len; + auto parse_result = duckdb_fast_float::from_chars(buf, buf + len, result, decimal_separator); + if (parse_result.ec != std::errc()) { + return false; + } + auto current_end = parse_result.ptr; + if (!strict) { + while (current_end < endptr && StringUtil::CharacterIsSpace(*current_end)) { + current_end++; + } + } + return current_end == endptr; +} +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/operator/integer_cast_operator.hpp b/src/duckdb/src/include/duckdb/common/operator/integer_cast_operator.hpp new file mode 100644 index 000000000..10953fe20 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/operator/integer_cast_operator.hpp @@ -0,0 +1,456 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/operator/integer_cast_operator.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/operator/add.hpp" +#include "duckdb/common/operator/multiply.hpp" +#include "duckdb/common/operator/subtract.hpp" +#include "duckdb/common/operator/cast_operators.hpp" + +namespace duckdb { +template +struct IntegerCastData { + using ResultType = T; + using StoreType = T; + ResultType result; +}; + +struct IntegerCastOperation { + template + static bool HandleDigit(T &state, uint8_t digit) { + using store_t = typename T::StoreType; + if (NEGATIVE) { + if (DUCKDB_UNLIKELY(state.result < (NumericLimits::Minimum() + digit) / 10)) { + return false; + } + state.result = state.result * 10 - digit; + } else { + if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 10)) { + return false; + } + state.result = state.result * 10 + digit; + } + return true; + } + + template + static bool HandleHexDigit(T &state, uint8_t digit) { + using store_t = typename T::StoreType; + if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 16)) { + return false; + } + state.result = state.result * 16 + digit; + return true; + } + + template + static bool HandleBinaryDigit(T &state, uint8_t digit) { + using store_t = typename T::StoreType; + if (DUCKDB_UNLIKELY(state.result > (NumericLimits::Maximum() - digit) / 2)) { + return false; + } + state.result = state.result * 2 + digit; + return true; + } + + template + static bool HandleExponent(T &state, int16_t exponent) { + // Simple integers don't deal with Exponents + return false; + } + + template + static bool HandleDecimal(T &state, uint8_t digit) { + // Simple integers don't deal with Decimals + return false; + } + + template + static bool Finalize(T &state) { + return true; + } +}; + +template +struct IntegerDecimalCastData { + using ResultType = T; + using StoreType = int64_t; + StoreType result; + StoreType decimal; + uint16_t decimal_digits; +}; + +template <> +struct IntegerDecimalCastData { + using ResultType = uint64_t; + using StoreType = uint64_t; + StoreType result; + StoreType decimal; + uint16_t decimal_digits; +}; + +struct IntegerDecimalCastOperation : IntegerCastOperation { + template + static bool HandleExponent(T &state, int16_t exponent) { + using store_t = typename T::StoreType; + + int16_t e = exponent; + // Negative Exponent + if (e < 0) { + while (state.result != 0 && e++ < 0) { + state.decimal = state.result % 10; + state.result /= 10; + } + if (state.decimal < 0) { + state.decimal = -state.decimal; + } + state.decimal_digits = 1; + return Finalize(state); + } + + // Positive Exponent + while (state.result != 0 && e-- > 0) { + if (!TryMultiplyOperator::Operation(state.result, (store_t)10, state.result)) { + return false; + } + } + + if (state.decimal == 0) { + return Finalize(state); + } + + // Handle decimals + e = exponent - state.decimal_digits; + store_t remainder = 0; + if (e < 0) { + if (static_cast(-e) <= NumericLimits::Digits()) { + store_t power = 1; + while (e++ < 0) { + power *= 10; + } + remainder = state.decimal % power; + state.decimal /= power; + } else { + state.decimal = 0; + } + } else { + while (e-- > 0) { + if (!TryMultiplyOperator::Operation(state.decimal, (store_t)10, state.decimal)) { + return false; + } + } + } + + state.decimal_digits -= exponent; + + if (NEGATIVE) { + if (!TrySubtractOperator::Operation(state.result, state.decimal, state.result)) { + return false; + } + } else if (!TryAddOperator::Operation(state.result, state.decimal, state.result)) { + return false; + } + state.decimal = remainder; + return Finalize(state); + } + + template + static bool HandleDecimal(T &state, uint8_t digit) { + using store_t = typename T::StoreType; + if (DUCKDB_UNLIKELY(state.decimal > (NumericLimits::Maximum() - digit) / 10)) { + // Simply ignore any more decimals + return true; + } + state.decimal_digits++; + state.decimal = state.decimal * 10 + digit; + return true; + } + + template + static bool Finalize(T &state) { + using result_t = typename T::ResultType; + using store_t = typename T::StoreType; + + result_t tmp; + if (!TryCast::Operation(state.result, tmp)) { + return false; + } + + while (state.decimal > 10) { + state.decimal /= 10; + state.decimal_digits--; + } + + bool success = true; + if (state.decimal_digits == 1 && state.decimal >= 5) { + if (NEGATIVE) { + success = TrySubtractOperator::Operation(tmp, (result_t)1, tmp); + } else { + success = TryAddOperator::Operation(tmp, (result_t)1, tmp); + } + } + state.result = tmp; + return success; + } +}; + +template +static bool IntegerCastLoop(const char *buf, idx_t len, T &result, bool strict) { + idx_t start_pos; + if (NEGATIVE) { + start_pos = 1; + } else { + if (*buf == '+') { + if (strict) { + // leading plus is not allowed in strict mode + return false; + } + start_pos = 1; + } else { + start_pos = 0; + } + } + idx_t pos = start_pos; + while (pos < len) { + if (!StringUtil::CharacterIsDigit(buf[pos])) { + // not a digit! + if (buf[pos] == decimal_separator) { + if (strict) { + return false; + } + bool number_before_period = pos > start_pos; + // decimal point: we accept decimal values for integers as well + // we just truncate them + // make sure everything after the period is a number + pos++; + idx_t start_digit = pos; + while (pos < len) { + if (!StringUtil::CharacterIsDigit(buf[pos])) { + break; + } + if (!OP::template HandleDecimal(result, buf[pos] - '0')) { + return false; + } + pos++; + + if (pos != len && buf[pos] == '_') { + // Skip one underscore if it is not the last character and followed by a digit + pos++; + if (pos == len || !StringUtil::CharacterIsDigit(buf[pos])) { + return false; + } + } + } + // make sure there is either (1) one number after the period, or (2) one number before the period + // i.e. we accept "1." and ".1" as valid numbers, but not "." + if (!(number_before_period || pos > start_digit)) { + return false; + } + if (pos >= len) { + break; + } + } + if (StringUtil::CharacterIsSpace(buf[pos])) { + // skip any trailing spaces + while (++pos < len) { + if (!StringUtil::CharacterIsSpace(buf[pos])) { + return false; + } + } + break; + } + if (ALLOW_EXPONENT) { + if (buf[pos] == 'e' || buf[pos] == 'E') { + if (pos == start_pos) { + return false; + } + pos++; + if (pos >= len) { + return false; + } + using ExponentData = IntegerCastData; + ExponentData exponent {}; + int negative = buf[pos] == '-'; + if (negative) { + if (!IntegerCastLoop( + buf + pos, len - pos, exponent, strict)) { + return false; + } + } else { + if (!IntegerCastLoop( + buf + pos, len - pos, exponent, strict)) { + return false; + } + } + return OP::template HandleExponent(result, exponent.result); + } + } + return false; + } + uint8_t digit = buf[pos++] - '0'; + if (!OP::template HandleDigit(result, digit)) { + return false; + } + + if (pos != len && buf[pos] == '_') { + // Skip one underscore if it is not the last character and followed by a digit + pos++; + if (pos == len || !StringUtil::CharacterIsDigit(buf[pos])) { + return false; + } + } + } + if (!OP::template Finalize(result)) { + return false; + } + return pos > start_pos; +} + +template +static bool IntegerHexCastLoop(const char *buf, idx_t len, T &result, bool strict) { + if (ALLOW_EXPONENT || NEGATIVE) { + return false; + } + idx_t start_pos = 1; + idx_t pos = start_pos; + char current_char; + while (pos < len) { + current_char = StringUtil::CharacterToLower(buf[pos]); + if (!StringUtil::CharacterIsHex(current_char)) { + return false; + } + uint8_t digit; + if (current_char >= 'a') { + digit = current_char - 'a' + 10; + } else { + digit = current_char - '0'; + } + pos++; + + if (pos != len && buf[pos] == '_') { + // Skip one underscore if it is not the last character and followed by a hex + pos++; + if (pos == len || !StringUtil::CharacterIsHex(buf[pos])) { + return false; + } + } + + if (!OP::template HandleHexDigit(result, digit)) { + return false; + } + } + if (!OP::template Finalize(result)) { + return false; + } + return pos > start_pos; +} + +template +static bool IntegerBinaryCastLoop(const char *buf, idx_t len, T &result, bool strict) { + if (ALLOW_EXPONENT || NEGATIVE) { + return false; + } + idx_t start_pos = 1; + idx_t pos = start_pos; + uint8_t digit; + char current_char; + while (pos < len) { + current_char = buf[pos]; + if (current_char == '0') { + digit = 0; + } else if (current_char == '1') { + digit = 1; + } else { + return false; + } + pos++; + if (pos != len && buf[pos] == '_') { + // Skip one underscore if it is not the last character and followed by a digit + pos++; + if (pos == len || (buf[pos] != '0' && buf[pos] != '1')) { + return false; + } + } + + if (!OP::template HandleBinaryDigit(result, digit)) { + return false; + } + } + if (!OP::template Finalize(result)) { + return false; + } + return pos > start_pos; +} + +template +static bool TryIntegerCast(const char *buf, idx_t len, T &result, bool strict) { + // skip any spaces at the start + while (len > 0 && StringUtil::CharacterIsSpace(*buf)) { + buf++; + len--; + } + if (len == 0) { + return false; + } + if (ZERO_INITIALIZE) { + memset(&result, 0, sizeof(T)); + } + // if the number is negative, we set the negative flag and skip the negative sign + if (*buf == '-') { + if (!IS_SIGNED) { + // Need to check if its not -0 + idx_t pos = 1; + while (pos < len) { + if (buf[pos++] != '0') { + return false; + } + } + } + return IntegerCastLoop(buf, len, result, strict); + } + if (len > 1 && *buf == '0') { + if (buf[1] == 'x' || buf[1] == 'X') { + // If it starts with 0x or 0X, we parse it as a hex value + buf++; + len--; + return IntegerHexCastLoop(buf, len, result, strict); + } else if (buf[1] == 'b' || buf[1] == 'B') { + // If it starts with 0b or 0B, we parse it as a binary value + buf++; + len--; + return IntegerBinaryCastLoop(buf, len, result, strict); + } else if (strict && StringUtil::CharacterIsDigit(buf[1])) { + // leading zeros are not allowed in strict mode + return false; + } + } + return IntegerCastLoop(buf, len, result, strict); +} + +template +static inline bool TrySimpleIntegerCast(const char *buf, idx_t len, T &result, bool strict) { + IntegerCastData simple_data; + if (TryIntegerCast, IS_SIGNED, false, IntegerCastOperation>(buf, len, simple_data, strict)) { + result = (T)simple_data.result; + return true; + } + + // Simple integer cast failed, try again with decimals/exponents included + // FIXME: This could definitely be improved as some extra work is being done here. It is more important that + // "normal" integers (without exponent/decimals) are still being parsed quickly. + IntegerDecimalCastData cast_data; + if (TryIntegerCast, IS_SIGNED, true, IntegerDecimalCastOperation>(buf, len, cast_data, + strict)) { + result = (T)cast_data.result; + return true; + } + return false; +} +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/operator/multiply.hpp b/src/duckdb/src/include/duckdb/common/operator/multiply.hpp index 5ee38f87a..0fad42bd5 100644 --- a/src/duckdb/src/include/duckdb/common/operator/multiply.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/multiply.hpp @@ -59,6 +59,8 @@ template <> DUCKDB_API bool TryMultiplyOperator::Operation(int64_t left, int64_t right, int64_t &result); template <> DUCKDB_API bool TryMultiplyOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t &result); +template <> +DUCKDB_API bool TryMultiplyOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result); struct MultiplyOperatorOverflowCheck { template diff --git a/src/duckdb/src/include/duckdb/common/operator/numeric_cast.hpp b/src/duckdb/src/include/duckdb/common/operator/numeric_cast.hpp index 3e356baf0..26603a987 100644 --- a/src/duckdb/src/include/duckdb/common/operator/numeric_cast.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/numeric_cast.hpp @@ -11,6 +11,7 @@ #include "duckdb/common/operator/cast_operators.hpp" #include "duckdb/common/types/bit.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/types/value.hpp" #include "duckdb/common/types/vector.hpp" @@ -263,6 +264,12 @@ bool TryCastWithOverflowCheck(hugeint_t input, bool &result) { return true; } +template <> +bool TryCastWithOverflowCheck(uhugeint_t input, bool &result) { + result = input.upper != 0 || input.lower != 0; + return true; +} + //===--------------------------------------------------------------------===// // Cast bool -> Numeric //===--------------------------------------------------------------------===// @@ -333,6 +340,13 @@ bool TryCastWithOverflowCheck(bool input, hugeint_t &result) { return true; } +template <> +bool TryCastWithOverflowCheck(bool input, uhugeint_t &result) { + result.upper = 0; + result.lower = input ? 1 : 0; + return true; +} + //===--------------------------------------------------------------------===// // Cast Numeric -> hugeint //===--------------------------------------------------------------------===// @@ -435,6 +449,11 @@ bool TryCastWithOverflowCheck(hugeint_t value, uint64_t &result) { return Hugeint::TryCast(value, result); } +template <> +bool TryCastWithOverflowCheck(hugeint_t value, uhugeint_t &result) { + return Hugeint::TryCast(value, result); +} + template <> bool TryCastWithOverflowCheck(hugeint_t value, float &result) { return Hugeint::TryCast(value, result); @@ -445,6 +464,123 @@ bool TryCastWithOverflowCheck(hugeint_t value, double &result) { return Hugeint::TryCast(value, result); } +//===--------------------------------------------------------------------===// +// Cast Uhugeint -> Numeric +//===--------------------------------------------------------------------===// +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, int8_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, int16_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, int32_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, int64_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, uint8_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, uint16_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, uint32_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, uint64_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, hugeint_t &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, float &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, double &result) { + return Uhugeint::TryCast(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uhugeint_t value, uhugeint_t &result) { + result = value; + return true; +} + +//===--------------------------------------------------------------------===// +// Cast Numeric -> uhugeint +//===--------------------------------------------------------------------===// +template <> +bool TryCastWithOverflowCheck(int8_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(int16_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(int32_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(int64_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uint8_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uint16_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uint32_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(uint64_t value, uhugeint_t &result) { + return Uhugeint::TryConvert(value, result); +} + +template <> +bool TryCastWithOverflowCheck(float value, uhugeint_t &result) { + return Uhugeint::TryConvert(std::nearbyintf(value), result); +} + +template <> +bool TryCastWithOverflowCheck(double value, uhugeint_t &result) { + return Uhugeint::TryConvert(std::nearbyint(value), result); +} + struct NumericTryCastToBit { template static inline string_t Operation(SRC input, Vector &result) { diff --git a/src/duckdb/src/include/duckdb/common/operator/string_cast.hpp b/src/duckdb/src/include/duckdb/common/operator/string_cast.hpp index 3e688f537..c436b0c6f 100644 --- a/src/duckdb/src/include/duckdb/common/operator/string_cast.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/string_cast.hpp @@ -46,6 +46,8 @@ DUCKDB_API duckdb::string_t StringCast::Operation(uint64_t input, Vector &result template <> DUCKDB_API duckdb::string_t StringCast::Operation(hugeint_t input, Vector &result); template <> +DUCKDB_API duckdb::string_t StringCast::Operation(uhugeint_t input, Vector &result); +template <> DUCKDB_API duckdb::string_t StringCast::Operation(float input, Vector &result); template <> DUCKDB_API duckdb::string_t StringCast::Operation(double input, Vector &result); diff --git a/src/duckdb/src/include/duckdb/common/operator/subtract.hpp b/src/duckdb/src/include/duckdb/common/operator/subtract.hpp index 7f268e711..0f39c976c 100644 --- a/src/duckdb/src/include/duckdb/common/operator/subtract.hpp +++ b/src/duckdb/src/include/duckdb/common/operator/subtract.hpp @@ -19,6 +19,7 @@ struct interval_t; struct date_t; struct timestamp_t; struct dtime_t; +struct dtime_tz_t; struct SubtractOperator { template @@ -38,7 +39,7 @@ int64_t SubtractOperator::Operation(date_t left, date_t right); template <> date_t SubtractOperator::Operation(date_t left, int32_t right); template <> -date_t SubtractOperator::Operation(date_t left, interval_t right); +timestamp_t SubtractOperator::Operation(date_t left, interval_t right); template <> timestamp_t SubtractOperator::Operation(timestamp_t left, interval_t right); template <> @@ -70,6 +71,8 @@ template <> bool TrySubtractOperator::Operation(int64_t left, int64_t right, int64_t &result); template <> bool TrySubtractOperator::Operation(hugeint_t left, hugeint_t right, hugeint_t &result); +template <> +bool TrySubtractOperator::Operation(uhugeint_t left, uhugeint_t right, uhugeint_t &result); struct SubtractOperatorOverflowCheck { template @@ -123,4 +126,7 @@ struct SubtractTimeOperator { template <> dtime_t SubtractTimeOperator::Operation(dtime_t left, interval_t right); +template <> +dtime_tz_t SubtractTimeOperator::Operation(dtime_tz_t left, interval_t right); + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/optional_idx.hpp b/src/duckdb/src/include/duckdb/common/optional_idx.hpp index 28c618f2a..7656fe6e7 100644 --- a/src/duckdb/src/include/duckdb/common/optional_idx.hpp +++ b/src/duckdb/src/include/duckdb/common/optional_idx.hpp @@ -25,15 +25,13 @@ class optional_idx { } static optional_idx Invalid() { - return INVALID_INDEX; + return optional_idx(); } bool IsValid() const { - return index != DConstants::INVALID_INDEX; - } - void Invalidate() { - index = INVALID_INDEX; + return index != INVALID_INDEX; } + idx_t GetIndex() const { if (index == INVALID_INDEX) { throw InternalException("Attempting to get the index of an optional_idx that is not set"); @@ -41,6 +39,10 @@ class optional_idx { return index; } + inline bool operator==(const optional_idx &rhs) const { + return index == rhs.index; + } + private: idx_t index; }; diff --git a/src/duckdb/src/include/duckdb/common/platform.h b/src/duckdb/src/include/duckdb/common/platform.h new file mode 100644 index 000000000..c7d2455a4 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/platform.h @@ -0,0 +1,53 @@ +#include +#include "duckdb/common/string_util.hpp" + +namespace duckdb { + +std::string DuckDBPlatform() { +#if defined(DUCKDB_CUSTOM_PLATFORM) + return DUCKDB_QUOTE_DEFINE(DUCKDB_CUSTOM_PLATFORM); +#endif +#if defined(DUCKDB_WASM_VERSION) + // DuckDB-Wasm requires CUSTOM_PLATFORM to be defined + static_assert(0, "DUCKDB_WASM_VERSION should rely on CUSTOM_PLATFORM being provided"); +#endif + std::string os = "linux"; +#if INTPTR_MAX == INT64_MAX + std::string arch = "amd64"; +#elif INTPTR_MAX == INT32_MAX + std::string arch = "i686"; +#else +#error Unknown pointer size or missing size macros! +#endif + std::string postfix = ""; + +#ifdef _WIN32 + os = "windows"; +#elif defined(__APPLE__) + os = "osx"; +#elif defined(__FreeBSD__) + os = "freebsd"; +#endif +#if defined(__aarch64__) || defined(__ARM_ARCH_ISA_A64) + arch = "arm64"; +#endif + +#if !defined(_GLIBCXX_USE_CXX11_ABI) || _GLIBCXX_USE_CXX11_ABI == 0 + if (os == "linux") { + postfix = "_gcc4"; + } +#endif +#if defined(__ANDROID__) + postfix += "_android"; // using + because it may also be gcc4 +#endif +#ifdef __MINGW32__ + postfix = "_mingw"; +#endif +// this is used for the windows R builds which use a separate build environment +#ifdef DUCKDB_PLATFORM_RTOOLS + postfix = "_rtools"; +#endif + return os + "_" + arch + postfix; +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/progress_bar/display/terminal_progress_bar_display.hpp b/src/duckdb/src/include/duckdb/common/progress_bar/display/terminal_progress_bar_display.hpp index d50434be1..98f6852c8 100644 --- a/src/duckdb/src/include/duckdb/common/progress_bar/display/terminal_progress_bar_display.hpp +++ b/src/duckdb/src/include/duckdb/common/progress_bar/display/terminal_progress_bar_display.hpp @@ -16,16 +16,15 @@ namespace duckdb { class TerminalProgressBarDisplay : public ProgressBarDisplay { public: - TerminalProgressBarDisplay() { - } - ~TerminalProgressBarDisplay() override { - } + TerminalProgressBarDisplay() = default; + ~TerminalProgressBarDisplay() override = default; public: void Update(double percentage) override; void Finish() override; private: + int32_t rendered_percentage = -1; static constexpr const idx_t PARTIAL_BLOCK_COUNT = UnicodeBar::PartialBlocksCount(); #ifndef DUCKDB_ASCII_TREE_RENDERER const char *PROGRESS_EMPTY = " "; @@ -43,7 +42,8 @@ class TerminalProgressBarDisplay : public ProgressBarDisplay { static constexpr const idx_t PROGRESS_BAR_WIDTH = 60; private: - void PrintProgressInternal(int percentage); + static int32_t NormalizePercentage(double percentage); + void PrintProgressInternal(int32_t percentage); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/radix.hpp b/src/duckdb/src/include/duckdb/common/radix.hpp index 46f6dc6d4..021d27186 100644 --- a/src/duckdb/src/include/duckdb/common/radix.hpp +++ b/src/duckdb/src/include/duckdb/common/radix.hpp @@ -165,6 +165,12 @@ inline void Radix::EncodeData(data_ptr_t dataptr, hugeint_t value) { EncodeData(dataptr + sizeof(value.upper), value.lower); } +template <> +inline void Radix::EncodeData(data_ptr_t dataptr, uhugeint_t value) { + EncodeData(dataptr, value.upper); + EncodeData(dataptr + sizeof(value.upper), value.lower); +} + template <> inline void Radix::EncodeData(data_ptr_t dataptr, float value) { uint32_t converted_value = EncodeFloat(value); diff --git a/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp index 9d8b11ecb..18f8a1573 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/binary_deserializer.hpp @@ -149,6 +149,7 @@ class BinaryDeserializer : public Deserializer { double ReadDouble() final; string ReadString() final; hugeint_t ReadHugeInt() final; + uhugeint_t ReadUhugeInt() final; void ReadDataPtr(data_ptr_t &ptr, idx_t count) final; }; diff --git a/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp index c7d22e311..0cf532b36 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/binary_serializer.hpp @@ -96,6 +96,7 @@ class BinarySerializer : public Serializer { void WriteValue(uint64_t value) final; void WriteValue(int64_t value) final; void WriteValue(hugeint_t value) final; + void WriteValue(uhugeint_t value) final; void WriteValue(float value) final; void WriteValue(double value) final; void WriteValue(const string_t value) final; diff --git a/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp index 8e455630b..000104c43 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/deserializer.hpp @@ -14,7 +14,8 @@ #include "duckdb/common/types/string_type.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/unordered_set.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/common/uhugeint.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" namespace duckdb { @@ -423,6 +424,12 @@ class Deserializer { return ReadHugeInt(); } + // Deserialize a uhugeint + template + inline typename std::enable_if::value, T>::type Read() { + return ReadUhugeInt(); + } + // Deserialize a LogicalIndex template inline typename std::enable_if::value, T>::type Read() { @@ -435,6 +442,13 @@ class Deserializer { return PhysicalIndex(ReadUnsignedInt64()); } + // Deserialize an optional_idx + template + inline typename std::enable_if::value, T>::type Read() { + auto idx = ReadUnsignedInt64(); + return idx == DConstants::INVALID_INDEX ? optional_idx() : optional_idx(idx); + } + protected: // Hooks for subclasses to override to implement custom behavior virtual void OnPropertyBegin(const field_id_t field_id, const char *tag) = 0; @@ -463,6 +477,7 @@ class Deserializer { virtual int64_t ReadSignedInt64() = 0; virtual uint64_t ReadUnsignedInt64() = 0; virtual hugeint_t ReadHugeInt() = 0; + virtual uhugeint_t ReadUhugeInt() = 0; virtual float ReadFloat() = 0; virtual double ReadDouble() = 0; virtual string ReadString() = 0; diff --git a/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp b/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp index 2fbce818a..0f60deecd 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/serialization_traits.hpp @@ -4,6 +4,7 @@ #include #include "duckdb/common/vector.hpp" +#include "duckdb/common/map.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/unordered_set.hpp" #include "duckdb/common/set.hpp" diff --git a/src/duckdb/src/include/duckdb/common/serializer/serializer.hpp b/src/duckdb/src/include/duckdb/common/serializer/serializer.hpp index 2520d4e01..10b926d04 100644 --- a/src/duckdb/src/include/duckdb/common/serializer/serializer.hpp +++ b/src/duckdb/src/include/duckdb/common/serializer/serializer.hpp @@ -12,8 +12,11 @@ #include "duckdb/common/serializer/serialization_traits.hpp" #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/string_type.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/unordered_set.hpp" +#include "duckdb/common/optional_idx.hpp" +#include "duckdb/common/value_operations/value_operations.hpp" namespace duckdb { @@ -276,6 +279,7 @@ class Serializer { virtual void WriteValue(uint64_t value) = 0; virtual void WriteValue(int64_t value) = 0; virtual void WriteValue(hugeint_t value) = 0; + virtual void WriteValue(uhugeint_t value) = 0; virtual void WriteValue(float value) = 0; virtual void WriteValue(double value) = 0; virtual void WriteValue(const string_t value) = 0; @@ -288,12 +292,20 @@ class Serializer { void WriteValue(PhysicalIndex value) { WriteValue(value.index); } + void WriteValue(optional_idx value) { + WriteValue(value.IsValid() ? value.GetIndex() : DConstants::INVALID_INDEX); + } }; // We need to special case vector because elements of vector cannot be referenced template <> void Serializer::WriteValue(const vector &vec); +// Specialization for Value (default Value comparison throws when comparing nulls) +template <> +void Serializer::WritePropertyWithDefault(const field_id_t field_id, const char *tag, const Value &value, + const Value &&default_value); + // List Impl template void Serializer::List::WriteObject(FUNC f) { diff --git a/src/duckdb/src/include/duckdb/common/string_util.hpp b/src/duckdb/src/include/duckdb/common/string_util.hpp index 1a563d88e..bef968e02 100644 --- a/src/duckdb/src/include/duckdb/common/string_util.hpp +++ b/src/duckdb/src/include/duckdb/common/string_util.hpp @@ -245,6 +245,18 @@ class StringUtil { } return strcmp(s1, s2) == 0; } + + //! JSON method that parses a { string: value } JSON blob + //! NOTE: this method ONLY parses a JSON {"key": "value"} object, it does not support ANYTHING else + //! NOTE: this method is not efficient + //! NOTE: this method is used in Exception construction - as such it does NOT throw on invalid JSON, instead an + //! empty map is returned + DUCKDB_API static unordered_map ParseJSONMap(const string &json); + //! JSON method that constructs a { string: value } JSON map + //! This is the inverse of ParseJSONMap + //! NOTE: this method is not efficient + DUCKDB_API static string ToJSONMap(ExceptionType type, const string &message, + const unordered_map &map); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/type_util.hpp b/src/duckdb/src/include/duckdb/common/type_util.hpp index fd8f40940..c198bafcd 100644 --- a/src/duckdb/src/include/duckdb/common/type_util.hpp +++ b/src/duckdb/src/include/duckdb/common/type_util.hpp @@ -13,6 +13,7 @@ #include "duckdb/common/types/datetime.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/types/interval.hpp" +#include "duckdb/common/uhugeint.hpp" namespace duckdb { @@ -39,6 +40,8 @@ PhysicalType GetTypeId() { return PhysicalType::UINT64; } else if (std::is_same()) { return PhysicalType::INT128; + } else if (std::is_same()) { + return PhysicalType::UINT128; } else if (std::is_same()) { return PhysicalType::INT32; } else if (std::is_same()) { @@ -60,7 +63,8 @@ PhysicalType GetTypeId() { template bool TypeIsNumber() { - return std::is_integral() || std::is_floating_point() || std::is_same(); + return std::is_integral() || std::is_floating_point() || std::is_same() || + std::is_same(); } template diff --git a/src/duckdb/src/include/duckdb/common/types.hpp b/src/duckdb/src/include/duckdb/common/types.hpp index 6bbd1bdca..bf6d23e08 100644 --- a/src/duckdb/src/include/duckdb/common/types.hpp +++ b/src/duckdb/src/include/duckdb/common/types.hpp @@ -41,10 +41,10 @@ struct list_entry_t { list_entry_t() = default; list_entry_t(uint64_t offset, uint64_t length) : offset(offset), length(length) { } - inline constexpr bool operator != (const list_entry_t &other) const { + inline constexpr bool operator!=(const list_entry_t &other) const { return !(*this == other); } - inline constexpr bool operator == (const list_entry_t &other) const { + inline constexpr bool operator==(const list_entry_t &other) const { return offset == other.offset && length == other.length; } @@ -52,7 +52,7 @@ struct list_entry_t { uint64_t length; }; -using union_tag_t = uint8_t; +using union_tag_t = uint8_t; //===--------------------------------------------------------------------===// // Internal Types @@ -61,7 +61,7 @@ using union_tag_t = uint8_t; // taken from arrow's type.h enum class PhysicalType : uint8_t { ///// A NULL type having no physical storage - //NA = 0, + // NA = 0, /// Boolean as 8 bit "bool" value BOOL = 1, @@ -91,7 +91,7 @@ enum class PhysicalType : uint8_t { INT64 = 9, ///// 2-byte floating point value - //HALF_FLOAT = 10, + // HALF_FLOAT = 10, /// 4-byte floating point value FLOAT = 11, @@ -100,31 +100,31 @@ enum class PhysicalType : uint8_t { DOUBLE = 12, ///// UTF8 variable-length string as List - //STRING = 13, + // STRING = 13, ///// Variable-length bytes (no guarantee of UTF8-ness) - //BINARY = 14, + // BINARY = 14, ///// Fixed-size binary. Each value occupies the same number of bytes - //FIXED_SIZE_BINARY = 15, + // FIXED_SIZE_BINARY = 15, ///// int32_t days since the UNIX epoch - //DATE32 = 16, + // DATE32 = 16, ///// int64_t milliseconds since the UNIX epoch - //DATE64 = 17, + // DATE64 = 17, ///// Exact timestamp encoded with int64 since UNIX epoch ///// Default unit millisecond - //TIMESTAMP = 18, + // TIMESTAMP = 18, ///// Time as signed 32-bit integer, representing either seconds or ///// milliseconds since midnight - //TIME32 = 19, + // TIME32 = 19, ///// Time as signed 64-bit integer, representing either microseconds or ///// nanoseconds since midnight - //TIME64 = 20, + // TIME64 = 20, /// YEAR_MONTH or DAY_TIME interval in SQL style INTERVAL = 21, @@ -140,35 +140,36 @@ enum class PhysicalType : uint8_t { STRUCT = 24, ///// Unions of logical types - //UNION = 25, + // UNION = 25, ///// Dictionary-encoded type, also called "categorical" or "factor" ///// in other programming languages. Holds the dictionary value ///// type but not the dictionary itself, which is part of the ///// ArrayData struct - //DICTIONARY = 26, + // DICTIONARY = 26, ///// Custom data type, implemented by user - //EXTENSION = 28, + // EXTENSION = 28, ///// 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. - //DURATION = 30, + // DURATION = 30, ///// Like STRING, but with 64-bit offsets - //LARGE_STRING = 31, + // LARGE_STRING = 31, ///// Like BINARY, but with 64-bit offsets - //LARGE_BINARY = 32, + // LARGE_BINARY = 32, ///// Like LIST, but with 64-bit offsets - //LARGE_LIST = 33, + // LARGE_LIST = 33, /// DuckDB Extensions VARCHAR = 200, // our own string representation, different from STRING and LARGE_STRING above + UINT128 = 203, // 128-bit unsigned integers INT128 = 204, // 128-bit integers UNKNOWN = 205, // Unknown physical type of user defined types /// Boolean as 1 bit, LSB bit-packed ordering @@ -185,7 +186,7 @@ enum class LogicalTypeId : uint8_t { SQLNULL = 1, /* NULL type, used for constant NULL */ UNKNOWN = 2, /* unknown type, used for parameter expressions */ ANY = 3, /* ANY type, used for functions that accept any type as parameter */ - USER = 4, /* A User Defined Type (e.g., ENUMs before the binder) */ + USER = 4, /* A User Defined Type (e.g., ENUMs before the binder) */ BOOLEAN = 10, TINYINT = 11, SMALLINT = 12, @@ -211,7 +212,10 @@ enum class LogicalTypeId : uint8_t { TIMESTAMP_TZ = 32, TIME_TZ = 34, BIT = 36, + STRING_LITERAL = 37, /* string literals, used for constant strings - only exists while binding */ + INTEGER_LITERAL = 38,/* integer literals, used for constant integers - only exists while binding */ + UHUGEINT = 49, HUGEINT = 50, POINTER = 51, VALIDITY = 53, @@ -228,7 +232,6 @@ enum class LogicalTypeId : uint8_t { ARRAY = 108 }; - struct ExtraTypeInfo; struct aggregate_state_t; @@ -256,20 +259,20 @@ struct LogicalType { return type_info_; } - inline void CopyAuxInfo(const LogicalType& other) { + inline void CopyAuxInfo(const LogicalType &other) { type_info_ = other.type_info_; } - bool EqualTypeInfo(const LogicalType& rhs) const; + bool EqualTypeInfo(const LogicalType &rhs) const; // copy assignment - inline LogicalType& operator=(const LogicalType &other) { + inline LogicalType &operator=(const LogicalType &other) { id_ = other.id_; physical_type_ = other.physical_type_; type_info_ = other.type_info_; return *this; } // move assignment - inline LogicalType& operator=(LogicalType&& other) noexcept { + inline LogicalType &operator=(LogicalType &&other) noexcept { id_ = other.id_; physical_type_ = other.physical_type_; std::swap(type_info_, other.type_info_); @@ -284,15 +287,12 @@ struct LogicalType { DUCKDB_API void Serialize(Serializer &serializer) const; DUCKDB_API static LogicalType Deserialize(Deserializer &deserializer); - static bool TypeIsTimestamp(LogicalTypeId id) { - return (id == LogicalTypeId::TIMESTAMP || - id == LogicalTypeId::TIMESTAMP_MS || - id == LogicalTypeId::TIMESTAMP_NS || - id == LogicalTypeId::TIMESTAMP_SEC || - id == LogicalTypeId::TIMESTAMP_TZ); + return (id == LogicalTypeId::TIMESTAMP || id == LogicalTypeId::TIMESTAMP_MS || + id == LogicalTypeId::TIMESTAMP_NS || id == LogicalTypeId::TIMESTAMP_SEC || + id == LogicalTypeId::TIMESTAMP_TZ); } - static bool TypeIsTimestamp(const LogicalType& type) { + static bool TypeIsTimestamp(const LogicalType &type) { return TypeIsTimestamp(type.id()); } DUCKDB_API string ToString() const; @@ -303,9 +303,16 @@ struct LogicalType { DUCKDB_API bool HasAlias() const; DUCKDB_API string GetAlias() const; - DUCKDB_API static LogicalType MaxLogicalType(const LogicalType &left, const LogicalType &right); + //! Returns the maximum logical type when combining the two types - or throws an exception if combining is not possible + DUCKDB_API static LogicalType MaxLogicalType(ClientContext &context, const LogicalType &left, const LogicalType &right); + DUCKDB_API static bool TryGetMaxLogicalType(ClientContext &context, const LogicalType &left, const LogicalType &right, LogicalType &result); + //! Forcibly returns a maximum logical type - similar to MaxLogicalType but never throws. As a fallback either left or right are returned. + DUCKDB_API static LogicalType ForceMaxLogicalType(const LogicalType &left, const LogicalType &right); + //! Normalize a type - removing literals + DUCKDB_API static LogicalType NormalizeType(const LogicalType &type); + - //! Gets the decimal properties of a numeric type. Fails if the type is not numeric. + //! Gets the decimal properties of a numeric type. Fails if the type is not numeric. DUCKDB_API bool GetDecimalProperties(uint8_t &width, uint8_t &scale) const; DUCKDB_API void Verify() const; @@ -345,9 +352,10 @@ struct LogicalType { static constexpr const LogicalTypeId VARCHAR = LogicalTypeId::VARCHAR; static constexpr const LogicalTypeId ANY = LogicalTypeId::ANY; static constexpr const LogicalTypeId BLOB = LogicalTypeId::BLOB; - static constexpr const LogicalTypeId BIT = LogicalTypeId::BIT; - static constexpr const LogicalTypeId INTERVAL = LogicalTypeId::INTERVAL; + static constexpr const LogicalTypeId BIT = LogicalTypeId::BIT; + static constexpr const LogicalTypeId INTERVAL = LogicalTypeId::INTERVAL; static constexpr const LogicalTypeId HUGEINT = LogicalTypeId::HUGEINT; + static constexpr const LogicalTypeId UHUGEINT = LogicalTypeId::UHUGEINT; static constexpr const LogicalTypeId UUID = LogicalTypeId::UUID; static constexpr const LogicalTypeId HASH = LogicalTypeId::UBIGINT; static constexpr const LogicalTypeId POINTER = LogicalTypeId::POINTER; @@ -359,20 +367,24 @@ struct LogicalType { // explicitly allowing these functions to be capitalized to be in-line with the remaining functions DUCKDB_API static LogicalType DECIMAL(int width, int scale); // NOLINT DUCKDB_API static LogicalType VARCHAR_COLLATION(string collation); // NOLINT - DUCKDB_API static LogicalType LIST(const LogicalType &child); // NOLINT + DUCKDB_API static LogicalType LIST(const LogicalType &child); // NOLINT DUCKDB_API static LogicalType STRUCT(child_list_t children); // NOLINT - DUCKDB_API static LogicalType AGGREGATE_STATE(aggregate_state_t state_type); // NOLINT - 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 + DUCKDB_API static LogicalType AGGREGATE_STATE(aggregate_state_t state_type); // NOLINT + 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 ARRAY(const LogicalType &child); // NOLINT DUCKDB_API static LogicalType ENUM(Vector &ordered_data, idx_t size); // NOLINT + // ANY but with special rules (default is LogicalType::ANY, 5) + DUCKDB_API static LogicalType ANY_PARAMS(LogicalType target, idx_t cast_score = 5); // NOLINT + //! Integer literal of the specified value + DUCKDB_API static LogicalType INTEGER_LITERAL(const Value &constant); // NOLINT // DEPRECATED - provided for backwards compatibility DUCKDB_API static LogicalType ENUM(const string &enum_name, Vector &ordered_data, idx_t size); // NOLINT - DUCKDB_API static LogicalType USER(const string &user_type_name); // NOLINT - DUCKDB_API static LogicalType USER(string catalog, string schema, string name); // NOLINT + DUCKDB_API static LogicalType USER(const string &user_type_name); // NOLINT + DUCKDB_API static LogicalType USER(string catalog, string schema, string name); // NOLINT //! A list of all NUMERIC types (integral and floating point types) DUCKDB_API static const vector Numeric(); //! A list of all INTEGRAL types @@ -381,6 +393,12 @@ struct LogicalType { DUCKDB_API static const vector Real(); //! A list of ALL SQL types DUCKDB_API static const vector AllTypes(); + +public: + //! The JSON type lives in the JSON extension, but we need to define this here for special handling + static constexpr auto JSON_TYPE_NAME = "JSON"; + DUCKDB_API static LogicalType JSON(); // NOLINT + DUCKDB_API bool IsJSONType() const; }; struct DecimalType { @@ -404,7 +422,7 @@ struct UserType { }; struct EnumType { - DUCKDB_API static int64_t GetPos(const LogicalType &type, const string_t& key); + DUCKDB_API static int64_t GetPos(const LogicalType &type, const string_t &key); DUCKDB_API static const Vector &GetValuesInsertOrder(const LogicalType &type); DUCKDB_API static idx_t GetSize(const LogicalType &type); DUCKDB_API static const string GetValue(const Value &val); @@ -416,6 +434,7 @@ struct StructType { DUCKDB_API static const child_list_t &GetChildTypes(const LogicalType &type); DUCKDB_API static const LogicalType &GetChildType(const LogicalType &type, idx_t index); DUCKDB_API static const string &GetChildName(const LogicalType &type, idx_t index); + DUCKDB_API static idx_t GetChildIndexUnsafe(const LogicalType &type, const string &name); DUCKDB_API static idx_t GetChildCount(const LogicalType &type); DUCKDB_API static bool IsUnnamed(const LogicalType &type); }; @@ -437,7 +456,7 @@ 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 + DUCKDB_API static constexpr idx_t MAX_ARRAY_SIZE = 100000; // 100k for now }; struct AggregateStateType { @@ -445,6 +464,18 @@ struct AggregateStateType { DUCKDB_API static const aggregate_state_t &GetStateType(const LogicalType &type); }; +struct AnyType { + DUCKDB_API static LogicalType GetTargetType(const LogicalType &type); + DUCKDB_API static idx_t GetCastScore(const LogicalType &type); +}; + +struct IntegerLiteral { + //! Returns the type that this integer literal "prefers" + DUCKDB_API static LogicalType GetType(const LogicalType &type); + //! Whether or not the integer literal fits into the target numeric type + DUCKDB_API static bool FitsInType(const LogicalType &type, const LogicalType &target); +}; + // **DEPRECATED**: Use EnumUtil directly instead. DUCKDB_API string LogicalTypeIdToString(LogicalTypeId type); @@ -468,8 +499,11 @@ bool ApproxEqual(float l, float r); bool ApproxEqual(double l, double r); struct aggregate_state_t { - aggregate_state_t() {} - aggregate_state_t(string function_name_p, LogicalType return_type_p, vector bound_argument_types_p) : function_name(std::move(function_name_p)), return_type(std::move(return_type_p)), bound_argument_types(std::move(bound_argument_types_p)) { + aggregate_state_t() { + } + aggregate_state_t(string function_name_p, LogicalType return_type_p, vector bound_argument_types_p) + : function_name(std::move(function_name_p)), return_type(std::move(return_type_p)), + bound_argument_types(std::move(bound_argument_types_p)) { } string function_name; diff --git a/src/duckdb/src/include/duckdb/common/types/cast_helpers.hpp b/src/duckdb/src/include/duckdb/common/types/cast_helpers.hpp index 77cec76d6..5d335ae19 100644 --- a/src/duckdb/src/include/duckdb/common/types/cast_helpers.hpp +++ b/src/duckdb/src/include/duckdb/common/types/cast_helpers.hpp @@ -13,6 +13,7 @@ #include "duckdb/common/types/decimal.hpp" #include "duckdb/common/types/interval.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/vector.hpp" #include "fmt/format.h" @@ -91,6 +92,9 @@ int NumericHelper::UnsignedLength(uint64_t value); template <> std::string NumericHelper::ToString(hugeint_t value); +template <> +std::string NumericHelper::ToString(uhugeint_t value); + struct DecimalToString { template static int DecimalLength(SIGNED value, uint8_t width, uint8_t scale) { @@ -345,6 +349,19 @@ struct HugeintToStringCast { } }; +struct UhugeintToStringCast { + static string_t Format(uhugeint_t value, Vector &vector) { + std::string str = value.ToString(); + string_t result = StringVector::EmptyString(vector, str.length()); + auto data = result.GetDataWriteable(); + + // null-termination not required + memcpy(data, str.data(), str.length()); + result.Finalize(); + return result; + } +}; + struct DateToStringCast { static idx_t Length(int32_t date[], idx_t &year_length, bool &add_bc) { // format is YYYY-MM-DD with optional (BC) at the end @@ -392,7 +409,7 @@ struct DateToStringCast { } // optionally add BC to the end of the date if (add_bc) { - memcpy(ptr, " (BC)", 5); + memcpy(ptr, " (BC)", 5); // NOLINT } } }; @@ -493,7 +510,7 @@ struct IntervalToStringCast { // append the name together with a potential "s" (for plurals) memcpy(buffer + length, name, name_len); length += name_len; - if (value != 1) { + if (value != 1 && value != -1) { buffer[length++] = 's'; } } @@ -554,7 +571,7 @@ struct IntervalToStringCast { } } else if (length == 0) { // empty interval: default to 00:00:00 - memcpy(buffer, "00:00:00", 8); + memcpy(buffer, "00:00:00", 8); // NOLINT return 8; } return length; diff --git a/src/duckdb/src/include/duckdb/common/types/chunk_collection.hpp b/src/duckdb/src/include/duckdb/common/types/chunk_collection.hpp deleted file mode 100644 index e3d1ddaa6..000000000 --- a/src/duckdb/src/include/duckdb/common/types/chunk_collection.hpp +++ /dev/null @@ -1,137 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/common/types/chunk_collection.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/common/enums/order_type.hpp" -#include "duckdb/common/types/data_chunk.hpp" -#include "duckdb/common/winapi.hpp" - -namespace duckdb { -class Allocator; -class ClientContext; - -//! A ChunkCollection represents a set of DataChunks that all have the same -//! types -/*! - A ChunkCollection represents a set of DataChunks concatenated together in a - list. Individual values of the collection can be iterated over using the - iterator. It is also possible to iterate directly over the chunks for more - direct access. -*/ -class ChunkCollection { -public: - explicit ChunkCollection(Allocator &allocator); - explicit ChunkCollection(ClientContext &context); - - //! The types of columns in the ChunkCollection - vector &Types() { - return types; - } - const vector &Types() const { - return types; - } - - //! The amount of rows in the ChunkCollection - const idx_t &Count() const { - return count; - } - - //! The amount of columns in the ChunkCollection - idx_t ColumnCount() const { - return types.size(); - } - - //! Append a new DataChunk directly to this ChunkCollection - DUCKDB_API void Append(DataChunk &new_chunk); - - //! Append a new DataChunk directly to this ChunkCollection - DUCKDB_API void Append(unique_ptr new_chunk); - - //! Append another ChunkCollection directly to this ChunkCollection - DUCKDB_API void Append(ChunkCollection &other); - - //! Merge is like Append but messes up the order and destroys the other collection - DUCKDB_API void Merge(ChunkCollection &other); - - //! Fuse adds new columns to the right of the collection - DUCKDB_API void Fuse(ChunkCollection &other); - - DUCKDB_API void Verify(); - - //! Gets the value of the column at the specified index - DUCKDB_API Value GetValue(idx_t column, idx_t index); - //! Sets the value of the column at the specified index - DUCKDB_API void SetValue(idx_t column, idx_t index, const Value &value); - - //! Copy a single cell to a target vector - DUCKDB_API void CopyCell(idx_t column, idx_t index, Vector &target, idx_t target_offset); - - DUCKDB_API string ToString() const; - DUCKDB_API void Print() const; - - //! Gets a reference to the chunk at the given index - DataChunk &GetChunkForRow(idx_t row_index) { - return *chunks[LocateChunk(row_index)]; - } - - //! Gets a reference to the chunk at the given index - DataChunk &GetChunk(idx_t chunk_index) { - D_ASSERT(chunk_index < chunks.size()); - return *chunks[chunk_index]; - } - const DataChunk &GetChunk(idx_t chunk_index) const { - D_ASSERT(chunk_index < chunks.size()); - return *chunks[chunk_index]; - } - - const vector> &Chunks() { - return chunks; - } - - idx_t ChunkCount() const { - return chunks.size(); - } - - void Reset() { - count = 0; - chunks.clear(); - types.clear(); - } - - unique_ptr Fetch() { - if (ChunkCount() == 0) { - return nullptr; - } - - auto res = std::move(chunks[0]); - chunks.erase(chunks.begin() + 0); - return res; - } - - //! Locates the chunk that belongs to the specific index - idx_t LocateChunk(idx_t index) { - idx_t result = index / STANDARD_VECTOR_SIZE; - D_ASSERT(result < chunks.size()); - return result; - } - - Allocator &GetAllocator() { - return allocator; - } - -private: - Allocator &allocator; - //! The total amount of elements in the collection - idx_t count; - //! The set of data chunks in the collection - vector> chunks; - //! The types of the ChunkCollection - vector types; -}; -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/types/datetime.hpp b/src/duckdb/src/include/duckdb/common/types/datetime.hpp index 5e13b6103..4a06e1b76 100644 --- a/src/duckdb/src/include/duckdb/common/types/datetime.hpp +++ b/src/duckdb/src/include/duckdb/common/types/datetime.hpp @@ -90,7 +90,7 @@ struct dtime_tz_t { // NOLINT static constexpr const int TIME_BITS = 40; static constexpr const int OFFSET_BITS = 24; static constexpr const uint64_t OFFSET_MASK = ~uint64_t(0) >> TIME_BITS; - static constexpr const int32_t MAX_OFFSET = 1559 * 60 * 60; + static constexpr const int32_t MAX_OFFSET = 16 * 60 * 60 - 1; // ±15:59:59 static constexpr const int32_t MIN_OFFSET = -MAX_OFFSET; uint64_t bits; @@ -98,7 +98,9 @@ struct dtime_tz_t { // NOLINT dtime_tz_t() = default; inline dtime_tz_t(dtime_t t, int32_t offset) - : bits((uint64_t(t.micros) << OFFSET_BITS) | uint64_t(offset + MAX_OFFSET)) { + : bits((uint64_t(t.micros) << OFFSET_BITS) | uint64_t(MAX_OFFSET - offset)) { + } + explicit inline dtime_tz_t(uint64_t bits_p) : bits(bits_p) { } inline dtime_t time() const { // NOLINT @@ -106,7 +108,7 @@ struct dtime_tz_t { // NOLINT } inline int32_t offset() const { // NOLINT - return int32_t(bits & OFFSET_MASK) - MAX_OFFSET; + return MAX_OFFSET - int32_t(bits & OFFSET_MASK); } // comparison operators diff --git a/src/duckdb/src/include/duckdb/common/types/hash.hpp b/src/duckdb/src/include/duckdb/common/types/hash.hpp index 337705a89..eeb849857 100644 --- a/src/duckdb/src/include/duckdb/common/types/hash.hpp +++ b/src/duckdb/src/include/duckdb/common/types/hash.hpp @@ -50,6 +50,8 @@ DUCKDB_API hash_t Hash(int64_t val); template <> DUCKDB_API hash_t Hash(hugeint_t val); template <> +DUCKDB_API hash_t Hash(uhugeint_t val); +template <> DUCKDB_API hash_t Hash(float val); template <> DUCKDB_API hash_t Hash(double val); diff --git a/src/duckdb/src/include/duckdb/common/types/hugeint.hpp b/src/duckdb/src/include/duckdb/common/types/hugeint.hpp index 23e6f1c6a..a5c06338e 100644 --- a/src/duckdb/src/include/duckdb/common/types/hugeint.hpp +++ b/src/duckdb/src/include/duckdb/common/types/hugeint.hpp @@ -40,38 +40,94 @@ class Hugeint { static hugeint_t Convert(T value) { hugeint_t result; if (!TryConvert(value, result)) { // LCOV_EXCL_START - throw ValueOutOfRangeException(double(value), GetTypeId(), GetTypeId()); + throw OutOfRangeException(double(value), GetTypeId(), GetTypeId()); } // LCOV_EXCL_STOP return result; } - static void NegateInPlace(hugeint_t &input) { - if (input.upper == NumericLimits::Minimum() && input.lower == 0) { - throw OutOfRangeException("HUGEINT is out of range"); + static bool TryNegate(hugeint_t input, hugeint_t &result); + + template + inline static void NegateInPlace(hugeint_t &input) { + if (!TryNegate(input, input)) { + throw OutOfRangeException("Negation of HUGEINT is out of range!"); } - input.lower = NumericLimits::Maximum() - input.lower + 1; - input.upper = -1 - input.upper + (input.lower == 0); } - static hugeint_t Negate(hugeint_t input) { - NegateInPlace(input); + + template + inline static hugeint_t Negate(hugeint_t input) { + NegateInPlace(input); return input; } static bool TryMultiply(hugeint_t lhs, hugeint_t rhs, hugeint_t &result); - static hugeint_t Add(hugeint_t lhs, hugeint_t rhs); - static hugeint_t Subtract(hugeint_t lhs, hugeint_t rhs); - static hugeint_t Multiply(hugeint_t lhs, hugeint_t rhs); - static hugeint_t Divide(hugeint_t lhs, hugeint_t rhs); - static hugeint_t Modulo(hugeint_t lhs, hugeint_t rhs); + template + inline static hugeint_t Multiply(hugeint_t lhs, hugeint_t rhs) { + hugeint_t result; + if (!TryMultiply(lhs, rhs, result)) { + throw OutOfRangeException("Overflow in HUGEINT multiplication: %s + %s", lhs.ToString(), rhs.ToString()); + } + return result; + } + + static bool TryDivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &result, hugeint_t &remainder); + + template + inline static hugeint_t Divide(hugeint_t lhs, hugeint_t rhs) { + // No division by zero + if (rhs == 0) { + throw OutOfRangeException("Division of HUGEINT by zero!"); + } + + // division only has one reason to overflow: MINIMUM / -1 + if (lhs == NumericLimits::Minimum() && rhs == -1) { + throw OutOfRangeException("Overflow in HUGEINT division: %s + %s", lhs.ToString(), rhs.ToString()); + } + return Divide(lhs, rhs); + } + + template + inline static hugeint_t Modulo(hugeint_t lhs, hugeint_t rhs) { + // No division by zero + if (rhs == 0) { + throw OutOfRangeException("Modulo of HUGEINT by zero: %s + %s", lhs.ToString(), rhs.ToString()); + } + + // division only has one reason to overflow: MINIMUM / -1 + if (lhs == NumericLimits::Minimum() && rhs == -1) { + throw OutOfRangeException("Overflow in HUGEINT modulo: %s + %s", lhs.ToString(), rhs.ToString()); + } + return Modulo(lhs, rhs); + } + + static bool TryAddInPlace(hugeint_t &lhs, hugeint_t rhs); + + template + inline static hugeint_t Add(hugeint_t lhs, hugeint_t rhs) { + if (!TryAddInPlace(lhs, rhs)) { + throw OutOfRangeException("Overflow in HUGEINT addition: %s + %s", lhs.ToString(), rhs.ToString()); + } + return lhs; + } + + static bool TrySubtractInPlace(hugeint_t &lhs, hugeint_t rhs); + + template + inline static hugeint_t Subtract(hugeint_t lhs, hugeint_t rhs) { + if (!TrySubtractInPlace(lhs, rhs)) { + throw OutOfRangeException("Underflow in HUGEINT addition: %s - %s", lhs.ToString(), rhs.ToString()); + } + return lhs; + } // DivMod -> returns the result of the division (lhs / rhs), and fills up the remainder static hugeint_t DivMod(hugeint_t lhs, hugeint_t rhs, hugeint_t &remainder); // DivMod but lhs MUST be positive, and rhs is a uint64_t static hugeint_t DivModPositive(hugeint_t lhs, uint64_t rhs, uint64_t &remainder); - static bool AddInPlace(hugeint_t &lhs, hugeint_t rhs); - static bool SubtractInPlace(hugeint_t &lhs, hugeint_t rhs); + static int Sign(hugeint_t n); + static hugeint_t Abs(hugeint_t n); // comparison operators // note that everywhere here we intentionally use bitwise ops @@ -81,35 +137,41 @@ class Hugeint { int upper_equals = lhs.upper == rhs.upper; return lower_equals & upper_equals; } + static bool NotEquals(hugeint_t lhs, hugeint_t rhs) { int lower_not_equals = lhs.lower != rhs.lower; int upper_not_equals = lhs.upper != rhs.upper; return lower_not_equals | upper_not_equals; } + static bool GreaterThan(hugeint_t lhs, hugeint_t rhs) { int upper_bigger = lhs.upper > rhs.upper; int upper_equal = lhs.upper == rhs.upper; int lower_bigger = lhs.lower > rhs.lower; return upper_bigger | (upper_equal & lower_bigger); } + static bool GreaterThanEquals(hugeint_t lhs, hugeint_t rhs) { int upper_bigger = lhs.upper > rhs.upper; int upper_equal = lhs.upper == rhs.upper; int lower_bigger_equals = lhs.lower >= rhs.lower; return upper_bigger | (upper_equal & lower_bigger_equals); } + static bool LessThan(hugeint_t lhs, hugeint_t rhs) { int upper_smaller = lhs.upper < rhs.upper; int upper_equal = lhs.upper == rhs.upper; int lower_smaller = lhs.lower < rhs.lower; return upper_smaller | (upper_equal & lower_smaller); } + static bool LessThanEquals(hugeint_t lhs, hugeint_t rhs) { int upper_smaller = lhs.upper < rhs.upper; int upper_equal = lhs.upper == rhs.upper; int lower_smaller_equals = lhs.lower <= rhs.lower; return upper_smaller | (upper_equal & lower_smaller_equals); } + static const hugeint_t POWERS_OF_TEN[40]; }; @@ -132,6 +194,8 @@ DUCKDB_API bool Hugeint::TryCast(hugeint_t input, uint64_t &result); template <> DUCKDB_API bool Hugeint::TryCast(hugeint_t input, hugeint_t &result); template <> +DUCKDB_API bool Hugeint::TryCast(hugeint_t input, uhugeint_t &result); +template <> DUCKDB_API bool Hugeint::TryCast(hugeint_t input, float &result); template <> DUCKDB_API bool Hugeint::TryCast(hugeint_t input, double &result); diff --git a/src/duckdb/src/include/duckdb/common/types/interval.hpp b/src/duckdb/src/include/duckdb/common/types/interval.hpp index f1786e217..a53f1ce7e 100644 --- a/src/duckdb/src/include/duckdb/common/types/interval.hpp +++ b/src/duckdb/src/include/duckdb/common/types/interval.hpp @@ -14,6 +14,7 @@ namespace duckdb { struct dtime_t; struct date_t; +struct dtime_tz_t; struct timestamp_t; class Serializer; @@ -150,6 +151,7 @@ class Interval { //! Add an interval to a time. In case the time overflows or underflows, modify the date by the overflow. //! For example if we go from 23:00 to 02:00, we add a day to the date static dtime_t Add(dtime_t left, interval_t right, date_t &date); + static dtime_tz_t Add(dtime_tz_t left, interval_t right, date_t &date); //! Comparison operators inline static bool Equals(const interval_t &left, const interval_t &right) { diff --git a/src/duckdb/src/include/duckdb/common/types/list_segment.hpp b/src/duckdb/src/include/duckdb/common/types/list_segment.hpp index ea4c2ad89..79f359faf 100644 --- a/src/duckdb/src/include/duckdb/common/types/list_segment.hpp +++ b/src/duckdb/src/include/duckdb/common/types/list_segment.hpp @@ -51,7 +51,7 @@ struct ListSegmentFunctions { void AppendRow(ArenaAllocator &allocator, LinkedList &linked_list, RecursiveUnifiedVectorFormat &input_data, idx_t &entry_idx) const; - void BuildListVector(const LinkedList &linked_list, Vector &result, idx_t &initial_total_count) const; + void BuildListVector(const LinkedList &linked_list, Vector &result, idx_t total_count) const; }; void GetSegmentDataFunctions(ListSegmentFunctions &functions, const LogicalType &type); diff --git a/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp b/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp index f31187232..de8f77c94 100644 --- a/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp +++ b/src/duckdb/src/include/duckdb/common/types/row/partitioned_tuple_data.hpp @@ -82,7 +82,7 @@ class PartitionedTupleData { //! Unpins the data void Unpin(); //! Get the partitions in this PartitionedTupleData - vector> &GetPartitions(); + unsafe_vector> &GetPartitions(); //! Get the data of this PartitionedTupleData as a single unpartitioned TupleDataCollection unique_ptr GetUnpartitioned(); //! Get the count of this PartitionedTupleData @@ -91,6 +91,8 @@ class PartitionedTupleData { idx_t SizeInBytes() const; //! Get the number of partitions of this PartitionedTupleData idx_t PartitionCount() const; + //! Get the count and size of the largest partition + void GetSizesAndCounts(vector &partition_sizes, vector &partition_counts) const; //! Converts this PartitionedTupleData to a string representation string ToString(); //! Prints the string representation of this PartitionedTupleData @@ -169,7 +171,7 @@ class PartitionedTupleData { mutex lock; shared_ptr allocators; - vector> partitions; + unsafe_vector> partitions; public: template diff --git a/src/duckdb/src/include/duckdb/common/types/string_type.hpp b/src/duckdb/src/include/duckdb/common/types/string_type.hpp index e0606506e..d6ab2c4bd 100644 --- a/src/duckdb/src/include/duckdb/common/types/string_type.hpp +++ b/src/duckdb/src/include/duckdb/common/types/string_type.hpp @@ -91,6 +91,10 @@ struct string_t { return value.inlined.length; } + bool Empty() const { + return value.inlined.length == 0; + } + string GetString() const { return string(GetData(), GetSize()); } @@ -113,9 +117,7 @@ struct string_t { // set trailing NULL byte if (GetSize() <= INLINE_LENGTH) { // fill prefix with zeros if the length is smaller than the prefix length - for (idx_t i = GetSize(); i < INLINE_BYTES; i++) { - value.inlined.inlined[i] = '\0'; - } + memset(value.inlined.inlined + GetSize(), 0, INLINE_BYTES - GetSize()); } else { // copy the data into the prefix #ifndef DUCKDB_DEBUG_NO_INLINE diff --git a/src/duckdb/src/include/duckdb/common/types/time.hpp b/src/duckdb/src/include/duckdb/common/types/time.hpp index a3042271d..e8cee119a 100644 --- a/src/duckdb/src/include/duckdb/common/types/time.hpp +++ b/src/duckdb/src/include/duckdb/common/types/time.hpp @@ -30,8 +30,6 @@ class Time { // No hour limit DUCKDB_API static bool TryConvertInterval(const char *buf, idx_t len, idx_t &pos, dtime_t &result, bool strict = false); - //! Format is ±[HH]HH[:MM[:SS]] - DUCKDB_API static bool TryParseUTCOffset(const char *str, idx_t &pos, idx_t len, int32_t &offset); //! Convert a time object to a string in the format "hh:mm:ss" DUCKDB_API static string ToString(dtime_t time); @@ -40,6 +38,9 @@ class Time { DUCKDB_API static dtime_t FromTime(int32_t hour, int32_t minute, int32_t second, int32_t microseconds = 0); + //! Normalize a TIME_TZ by adding the offset to the time part and returning the TIME + DUCKDB_API static dtime_t NormalizeTimeTZ(dtime_tz_t timetz); + //! Extract the time from a given timestamp object DUCKDB_API static void Convert(dtime_t time, int32_t &out_hour, int32_t &out_min, int32_t &out_sec, int32_t &out_micros); diff --git a/src/duckdb/src/include/duckdb/common/types/timestamp.hpp b/src/duckdb/src/include/duckdb/common/types/timestamp.hpp index 0bb5101a4..0d0851fa3 100644 --- a/src/duckdb/src/include/duckdb/common/types/timestamp.hpp +++ b/src/duckdb/src/include/duckdb/common/types/timestamp.hpp @@ -20,6 +20,7 @@ namespace duckdb { struct date_t; struct dtime_t; +struct dtime_tz_t; //! Type used to represent timestamps (seconds,microseconds,milliseconds or nanoseconds since 1970-01-01) struct timestamp_t { // NOLINT @@ -117,6 +118,7 @@ class Timestamp { //! Create a Timestamp object from a specified (date, time) combination DUCKDB_API static timestamp_t FromDatetime(date_t date, dtime_t time); DUCKDB_API static bool TryFromDatetime(date_t date, dtime_t time, timestamp_t &result); + DUCKDB_API static bool TryFromDatetime(date_t date, dtime_tz_t timetz, timestamp_t &result); //! Is the character a valid part of a time zone name? static inline bool CharacterIsTimeZone(char c) { diff --git a/src/duckdb/src/include/duckdb/common/types/uhugeint.hpp b/src/duckdb/src/include/duckdb/common/types/uhugeint.hpp new file mode 100644 index 000000000..335feef79 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/types/uhugeint.hpp @@ -0,0 +1,216 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/common/types/uhugeint.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/types.hpp" +#include "duckdb/common/type_util.hpp" +#include "duckdb/common/limits.hpp" +#include "duckdb/common/exception.hpp" +#include "duckdb/common/uhugeint.hpp" + +namespace duckdb { + +//! The uhugeint class contains static operations for the UINT128 type +class Uhugeint { +public: + //! Convert a uhugeint object to a string + static string ToString(uhugeint_t input); + + template + DUCKDB_API static bool TryCast(uhugeint_t input, T &result); + + template + static T Cast(uhugeint_t input) { + T result = 0; + TryCast(input, result); + return result; + } + + template + static bool TryConvert(T value, uhugeint_t &result); + + template + static uhugeint_t Convert(T value) { + uhugeint_t result; + if (!TryConvert(value, result)) { // LCOV_EXCL_START + throw OutOfRangeException(double(value), GetTypeId(), GetTypeId()); + } // LCOV_EXCL_STOP + return result; + } + + static bool TryNegate(uhugeint_t input, uhugeint_t &result); + + // "The negative of an unsigned quantity is computed by subtracting its value from 2^n, where n is the number of + // bits in the promoted operand." + template + inline static void NegateInPlace(uhugeint_t &input) { + if (!TryNegate(input, input)) { + throw OutOfRangeException("UHUGEINT is out of range"); + } + } + + template + inline static uhugeint_t Negate(uhugeint_t input) { + NegateInPlace(input); + return input; + } + + static bool TryMultiply(uhugeint_t lhs, uhugeint_t rhs, uhugeint_t &result); + + template + inline static uhugeint_t Multiply(uhugeint_t lhs, uhugeint_t rhs) { + uhugeint_t result; + if (!TryMultiply(lhs, rhs, result)) { + throw OutOfRangeException("Overflow in UHUGEINT multiplication!: %s + %s", lhs.ToString(), rhs.ToString()); + } + return result; + } + + static bool TryDivMod(uhugeint_t lhs, uhugeint_t rhs, uhugeint_t &result, uhugeint_t &remainder); + + template + inline static uhugeint_t Divide(uhugeint_t lhs, uhugeint_t rhs) { + // division between two same-size unsigned intergers can only go wrong with division by zero + if (rhs == 0) { + throw OutOfRangeException("Division of UHUGEINT by zero!"); + } + return Divide(lhs, rhs); + } + + template + inline static uhugeint_t Modulo(uhugeint_t lhs, uhugeint_t rhs) { + if (rhs == 0) { + throw OutOfRangeException("Modulo of UHUGEINT by zero!"); + } + return Modulo(lhs, rhs); + } + + static bool TryAddInPlace(uhugeint_t &lhs, uhugeint_t rhs); + + template + inline static uhugeint_t Add(uhugeint_t lhs, uhugeint_t rhs) { + if (!TryAddInPlace(lhs, rhs)) { + throw OutOfRangeException("Overflow in UHUGEINT addition: %s + %s", lhs.ToString(), rhs.ToString()); + } + return lhs; + } + + static bool TrySubtractInPlace(uhugeint_t &lhs, uhugeint_t rhs); + + template + inline static uhugeint_t Subtract(uhugeint_t lhs, uhugeint_t rhs) { + if (!TrySubtractInPlace(lhs, rhs)) { + throw OutOfRangeException("Underflow in HUGEINT addition: %s - %s", lhs.ToString(), rhs.ToString()); + } + return lhs; + } + + // DivMod -> returns the result of the division (lhs / rhs), and fills up the remainder + static uhugeint_t DivMod(uhugeint_t lhs, uhugeint_t rhs, uhugeint_t &remainder); + + static int Sign(hugeint_t n); + static hugeint_t Abs(hugeint_t n); + + // comparison operators + // note that everywhere here we intentionally use bitwise ops + // this is because they seem to be consistently much faster (benchmarked on a Macbook Pro) + static bool Equals(uhugeint_t lhs, uhugeint_t rhs) { + int lower_equals = lhs.lower == rhs.lower; + int upper_equals = lhs.upper == rhs.upper; + return lower_equals & upper_equals; + } + + static bool NotEquals(uhugeint_t lhs, uhugeint_t rhs) { + int lower_not_equals = lhs.lower != rhs.lower; + int upper_not_equals = lhs.upper != rhs.upper; + return lower_not_equals | upper_not_equals; + } + + static bool GreaterThan(uhugeint_t lhs, uhugeint_t rhs) { + int upper_bigger = lhs.upper > rhs.upper; + int upper_equal = lhs.upper == rhs.upper; + int lower_bigger = lhs.lower > rhs.lower; + return upper_bigger | (upper_equal & lower_bigger); + } + + static bool GreaterThanEquals(uhugeint_t lhs, uhugeint_t rhs) { + int upper_bigger = lhs.upper > rhs.upper; + int upper_equal = lhs.upper == rhs.upper; + int lower_bigger_equals = lhs.lower >= rhs.lower; + return upper_bigger | (upper_equal & lower_bigger_equals); + } + + static bool LessThan(uhugeint_t lhs, uhugeint_t rhs) { + int upper_smaller = lhs.upper < rhs.upper; + int upper_equal = lhs.upper == rhs.upper; + int lower_smaller = lhs.lower < rhs.lower; + return upper_smaller | (upper_equal & lower_smaller); + } + + static bool LessThanEquals(uhugeint_t lhs, uhugeint_t rhs) { + int upper_smaller = lhs.upper < rhs.upper; + int upper_equal = lhs.upper == rhs.upper; + int lower_smaller_equals = lhs.lower <= rhs.lower; + return upper_smaller | (upper_equal & lower_smaller_equals); + } + + static const uhugeint_t POWERS_OF_TEN[40]; +}; + +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, int8_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, int16_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, int32_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, int64_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, uint8_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, uint16_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, uint32_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, uint64_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, hugeint_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, uhugeint_t &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, float &result); +template <> +DUCKDB_API bool Uhugeint::TryCast(uhugeint_t input, double &result); + +template <> +bool Uhugeint::TryConvert(int8_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(int16_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(int32_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(int64_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(uint8_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(uint16_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(uint32_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(uint64_t value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(float value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(double value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(long double value, uhugeint_t &result); +template <> +bool Uhugeint::TryConvert(const char *value, uhugeint_t &result); + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/types/value.hpp b/src/duckdb/src/include/duckdb/common/types/value.hpp index 22759c703..fd1f91811 100644 --- a/src/duckdb/src/include/duckdb/common/types/value.hpp +++ b/src/duckdb/src/include/duckdb/common/types/value.hpp @@ -11,6 +11,7 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/types.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/winapi.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/types/date.hpp" @@ -84,6 +85,7 @@ class Value { //! Create a Numeric value of the specified type with the specified value DUCKDB_API static Value Numeric(const LogicalType &type, int64_t value); DUCKDB_API static Value Numeric(const LogicalType &type, hugeint_t value); + DUCKDB_API static Value Numeric(const LogicalType &type, uhugeint_t value); //! Create a tinyint Value from a specified value DUCKDB_API static Value BOOLEAN(int8_t value); @@ -105,6 +107,8 @@ class Value { DUCKDB_API static Value UBIGINT(uint64_t value); //! Create a hugeint Value from a specified value DUCKDB_API static Value HUGEINT(hugeint_t value); + //! Create a uhugeint Value from a specified value + DUCKDB_API static Value UHUGEINT(uhugeint_t value); //! Create a uuid Value from a specified value DUCKDB_API static Value UUID(const string &value); //! Create a uuid Value from a specified value @@ -304,6 +308,7 @@ class Value { uint32_t uinteger; uint64_t ubigint; hugeint_t hugeint; + uhugeint_t uhugeint; float float_; // NOLINT double double_; // NOLINT uintptr_t pointer; @@ -367,6 +372,10 @@ struct UBigIntValue { DUCKDB_API static uint64_t Get(const Value &value); }; +struct UhugeIntValue { + DUCKDB_API static uhugeint_t Get(const Value &value); +}; + struct FloatValue { DUCKDB_API static float Get(const Value &value); }; @@ -440,6 +449,8 @@ Value DUCKDB_API Value::CreateValue(int64_t value); template <> Value DUCKDB_API Value::CreateValue(hugeint_t value); template <> +Value DUCKDB_API Value::CreateValue(uhugeint_t value); +template <> Value DUCKDB_API Value::CreateValue(date_t value); template <> Value DUCKDB_API Value::CreateValue(dtime_t value); @@ -491,6 +502,8 @@ DUCKDB_API uint64_t Value::GetValue() const; template <> DUCKDB_API hugeint_t Value::GetValue() const; template <> +DUCKDB_API uhugeint_t Value::GetValue() const; +template <> DUCKDB_API string Value::GetValue() const; template <> DUCKDB_API float Value::GetValue() const; @@ -520,6 +533,8 @@ DUCKDB_API int64_t Value::GetValueUnsafe() const; template <> DUCKDB_API hugeint_t Value::GetValueUnsafe() const; template <> +DUCKDB_API uhugeint_t Value::GetValueUnsafe() const; +template <> DUCKDB_API uint8_t Value::GetValueUnsafe() const; template <> DUCKDB_API uint16_t Value::GetValueUnsafe() const; diff --git a/src/duckdb/src/include/duckdb/common/types/vector.hpp b/src/duckdb/src/include/duckdb/common/types/vector.hpp index 1f6b56776..bf5d339e0 100644 --- a/src/duckdb/src/include/duckdb/common/types/vector.hpp +++ b/src/duckdb/src/include/duckdb/common/types/vector.hpp @@ -118,9 +118,9 @@ class Vector { DUCKDB_API void ResetFromCache(const VectorCache &cache); //! Creates a reference to a slice of the other vector - DUCKDB_API void Slice(Vector &other, idx_t offset, idx_t end); + DUCKDB_API void Slice(const Vector &other, idx_t offset, idx_t end); //! Creates a reference to a slice of the other vector - DUCKDB_API void Slice(Vector &other, const SelectionVector &sel, idx_t count); + DUCKDB_API void Slice(const Vector &other, const SelectionVector &sel, idx_t count); //! Turns the vector into a dictionary vector with the specified dictionary DUCKDB_API void Slice(const SelectionVector &sel, idx_t count); //! Slice the vector, keeping the result around in a cache or potentially using the cache instead of slicing diff --git a/src/duckdb/src/include/duckdb/common/uhugeint.hpp b/src/duckdb/src/include/duckdb/common/uhugeint.hpp new file mode 100644 index 000000000..fdb1e9319 --- /dev/null +++ b/src/duckdb/src/include/duckdb/common/uhugeint.hpp @@ -0,0 +1,81 @@ +#pragma once + +#include "duckdb/common/winapi.hpp" +#include "duckdb/common/string.hpp" +#include + +namespace duckdb { + +// Forward declaration to allow conversion between hugeint and uhugeint +struct hugeint_t; + +struct uhugeint_t { +public: + uint64_t lower; + uint64_t upper; + +public: + uhugeint_t() = default; + DUCKDB_API uhugeint_t(uint64_t value); // NOLINT: Allow implicit conversion from `uint64_t` + constexpr uhugeint_t(uint64_t upper, uint64_t lower) : lower(lower), upper(upper) { + } + constexpr uhugeint_t(const uhugeint_t &rhs) = default; + constexpr uhugeint_t(uhugeint_t &&rhs) = default; + uhugeint_t &operator=(const uhugeint_t &rhs) = default; + uhugeint_t &operator=(uhugeint_t &&rhs) = default; + + DUCKDB_API string ToString() const; + + // comparison operators + DUCKDB_API bool operator==(const uhugeint_t &rhs) const; + DUCKDB_API bool operator!=(const uhugeint_t &rhs) const; + DUCKDB_API bool operator<=(const uhugeint_t &rhs) const; + DUCKDB_API bool operator<(const uhugeint_t &rhs) const; + DUCKDB_API bool operator>(const uhugeint_t &rhs) const; + DUCKDB_API bool operator>=(const uhugeint_t &rhs) const; + + // arithmetic operators + DUCKDB_API uhugeint_t operator+(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator-(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator*(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator/(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator%(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator-() const; + + // bitwise operators + DUCKDB_API uhugeint_t operator>>(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator<<(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator&(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator|(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator^(const uhugeint_t &rhs) const; + DUCKDB_API uhugeint_t operator~() const; + + // in-place operators + DUCKDB_API uhugeint_t &operator+=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator-=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator*=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator/=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator%=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator>>=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator<<=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator&=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator|=(const uhugeint_t &rhs); + DUCKDB_API uhugeint_t &operator^=(const uhugeint_t &rhs); + + // boolean operators + DUCKDB_API explicit operator bool() const; + DUCKDB_API bool operator!() const; + + // cast operators -- doesn't check bounds/overflow/underflow + DUCKDB_API explicit operator uint8_t() const; + DUCKDB_API explicit operator uint16_t() const; + DUCKDB_API explicit operator uint32_t() const; + DUCKDB_API explicit operator uint64_t() const; + DUCKDB_API explicit operator int8_t() const; + DUCKDB_API explicit operator int16_t() const; + DUCKDB_API explicit operator int32_t() const; + DUCKDB_API explicit operator int64_t() const; + DUCKDB_API operator hugeint_t() const; // NOLINT: Allow implicit conversion from `uhugeint_t` +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/common/vector_size.hpp b/src/duckdb/src/include/duckdb/common/vector_size.hpp index db28133a1..0fa29086b 100644 --- a/src/duckdb/src/include/duckdb/common/vector_size.hpp +++ b/src/duckdb/src/include/duckdb/common/vector_size.hpp @@ -12,16 +12,16 @@ namespace duckdb { +//! The default standard vector size +#define DEFAULT_STANDARD_VECTOR_SIZE 2048 + //! The vector size used in the execution engine #ifndef STANDARD_VECTOR_SIZE -#define STANDARD_VECTOR_SIZE 2048 +#define STANDARD_VECTOR_SIZE DEFAULT_STANDARD_VECTOR_SIZE #endif -#if ((STANDARD_VECTOR_SIZE & (STANDARD_VECTOR_SIZE - 1)) != 0) -#error Vector size should be a power of two +#if (STANDARD_VECTOR_SIZE & (STANDARD_VECTOR_SIZE - 1) != 0) +#error The vector size must be a power of two #endif -//! Zero selection vector: completely filled with the value 0 [READ ONLY] -extern const sel_t ZERO_VECTOR[STANDARD_VECTOR_SIZE]; - } // namespace duckdb 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..0510f76a7 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 @@ -27,7 +27,7 @@ struct ApproxCountDistinctFun { struct ArgMinFun { static constexpr const char *Name = "arg_min"; static constexpr const char *Parameters = "arg,val"; - static constexpr const char *Description = "Finds the row with the minimum val. Calculates the arg expression at that row."; + static constexpr const char *Description = "Finds the row with the minimum val. Calculates the non-NULL arg expression at that row."; static constexpr const char *Example = "arg_min(A,B)"; static AggregateFunctionSet GetFunctions(); @@ -45,10 +45,19 @@ struct MinByFun { static constexpr const char *Name = "min_by"; }; +struct ArgMinNullFun { + static constexpr const char *Name = "arg_min_null"; + static constexpr const char *Parameters = "arg,val"; + static constexpr const char *Description = "Finds the row with the minimum val. Calculates the arg expression at that row."; + static constexpr const char *Example = "arg_min_null(A,B)"; + + static AggregateFunctionSet GetFunctions(); +}; + struct ArgMaxFun { static constexpr const char *Name = "arg_max"; static constexpr const char *Parameters = "arg,val"; - static constexpr const char *Description = "Finds the row with the maximum val. Calculates the arg expression at that row."; + static constexpr const char *Description = "Finds the row with the maximum val. Calculates the non-NULL arg expression at that row."; static constexpr const char *Example = "arg_max(A,B)"; static AggregateFunctionSet GetFunctions(); @@ -66,6 +75,15 @@ struct MaxByFun { static constexpr const char *Name = "max_by"; }; +struct ArgMaxNullFun { + static constexpr const char *Name = "arg_max_null"; + static constexpr const char *Parameters = "arg,val"; + static constexpr const char *Description = "Finds the row with the maximum val. Calculates the arg expression at that row."; + static constexpr const char *Example = "arg_max_null(A,B)"; + + static AggregateFunctionSet GetFunctions(); +}; + struct BitAndFun { static constexpr const char *Name = "bit_and"; static constexpr const char *Parameters = "arg"; @@ -159,6 +177,15 @@ struct KurtosisFun { static AggregateFunction GetFunction(); }; +struct KurtosisPopFun { + static constexpr const char *Name = "kurtosis_pop"; + static constexpr const char *Parameters = "x"; + static constexpr const char *Description = "Returns the excess kurtosis (Fisher’s definition) of all input values, without bias correction"; + static constexpr const char *Example = ""; + + static AggregateFunction GetFunction(); +}; + struct MinFun { static constexpr const char *Name = "min"; static constexpr const char *Parameters = "arg"; diff --git a/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp index 6571f0f9e..f560bf4ba 100644 --- a/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/lambda_functions.hpp @@ -12,6 +12,9 @@ #include "duckdb/execution/expression_executor_state.hpp" #include "duckdb/execution/expression_executor.hpp" +#include "duckdb/planner/expression/bound_function_expression.hpp" +#include "duckdb/planner/expression/bound_cast_expression.hpp" + namespace duckdb { struct ListLambdaBindData : public FunctionData { @@ -41,6 +44,13 @@ 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 parameter type for ternary lambdas + static LogicalType BindTernaryLambda(const idx_t parameter_idx, const LogicalType &list_child_type); + + //! Checks for NULL list parameter and prepared statements and adds bound cast expression + static unique_ptr ListLambdaPrepareBind(vector> &arguments, + ClientContext &context, ScalarFunction &bound_function); + //! Returns the ListLambdaBindData containing the lambda expression static unique_ptr ListLambdaBind(ClientContext &, ScalarFunction &bound_function, vector> &arguments, @@ -50,6 +60,72 @@ class LambdaFunctions { static void ListTransformFunction(DataChunk &args, ExpressionState &state, Vector &result); //! Internally executes list_filter static void ListFilterFunction(DataChunk &args, ExpressionState &state, Vector &result); + //! Internally executes list_reduce + static void ListReduceFunction(DataChunk &args, ExpressionState &state, Vector &result); + +public: + //! Lambda expressions can only be executed on one STANDARD_VECTOR_SIZE list child elements at a time, so for + //! list_transform and list_filter we need to prepare the input vectors for the lambda expression. In list_reduce + //! the input size can never exceed row_count so it doesn't need ColumnInfo. + struct ColumnInfo { + explicit ColumnInfo(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; + }; + + //! LambdaInfo sets up and stores the information needed by all lambda functions + struct LambdaInfo { + explicit LambdaInfo(DataChunk &args, ExpressionState &state, Vector &result, bool &result_is_null) + : result(result), row_count(args.size()), is_all_constant(args.AllConstant()) { + Vector &list_column = args.data[0]; + + result.SetVectorType(VectorType::FLAT_VECTOR); + result_validity = &FlatVector::Validity(result); + + if (list_column.GetType().id() == LogicalTypeId::SQLNULL) { + result_validity->SetInvalid(0); + result_is_null = true; + return; + } + + // get the lambda expression + auto &func_expr = state.expr.Cast(); + auto &bind_info = func_expr.bind_info->Cast(); + lambda_expr = bind_info.lambda_expr; + is_volatile = lambda_expr->IsVolatile(); + has_index = bind_info.has_index; + + // get the list column entries + list_column.ToUnifiedFormat(row_count, list_column_format); + list_entries = UnifiedVectorFormat::GetData(list_column_format); + + child_vector = &ListVector::GetEntry(list_column); + + // get the lambda column data for all other input vectors + column_infos = LambdaFunctions::GetColumnInfo(args, row_count); + }; + + const list_entry_t *list_entries; + UnifiedVectorFormat list_column_format; + optional_ptr child_vector; + Vector &result; + optional_ptr result_validity; + vector column_infos; + optional_ptr lambda_expr; + + const idx_t row_count; + bool has_index; + bool is_volatile; + const bool is_all_constant; + }; + + static vector GetColumnInfo(DataChunk &args, const idx_t row_count); + static vector> GetInconstantColumnInfo(vector &data); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/blob_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/blob_functions.hpp index 0b1c94832..29c4034f4 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/blob_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/blob_functions.hpp @@ -57,4 +57,13 @@ struct Base64Fun { static constexpr const char *Name = "base64"; }; +struct CreateSortKeyFun { + static constexpr const char *Name = "create_sort_key"; + static constexpr const char *Parameters = "parameters..."; + static constexpr const char *Description = "Constructs a binary-comparable sort key based on a set of input parameters and sort qualifiers"; + static constexpr const char *Example = "create_sort_key('A', 'DESC')"; + + static ScalarFunction GetFunction(); +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/date_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/date_functions.hpp index 67c282516..2074da5f0 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/date_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/date_functions.hpp @@ -464,9 +464,9 @@ struct ToDecadesFun { struct ToHoursFun { static constexpr const char *Name = "to_hours"; - static constexpr const char *Parameters = "double"; + static constexpr const char *Parameters = "integer"; static constexpr const char *Description = "Construct a hour interval"; - static constexpr const char *Example = "to_hours(5.5)"; + static constexpr const char *Example = "to_hours(5)"; static ScalarFunction GetFunction(); }; @@ -500,9 +500,9 @@ struct ToMillisecondsFun { struct ToMinutesFun { static constexpr const char *Name = "to_minutes"; - static constexpr const char *Parameters = "double"; + static constexpr const char *Parameters = "integer"; static constexpr const char *Description = "Construct a minute interval"; - static constexpr const char *Example = "to_minutes(5.5)"; + static constexpr const char *Example = "to_minutes(5)"; static ScalarFunction GetFunction(); }; 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 a3bf36976..f37bce788 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 @@ -222,6 +222,27 @@ struct FilterFun { static constexpr const char *Name = "filter"; }; +struct ListReduceFun { + static constexpr const char *Name = "list_reduce"; + static constexpr const char *Parameters = "list,lambda"; + static constexpr const char *Description = "Returns a single value that is the result of applying the lambda function to each element of the input list, starting with the first element and then repeatedly applying the lambda function to the result of the previous application and the next element of the list."; + static constexpr const char *Example = "list_reduce([1, 2, 3], (x, y) -> x + y)"; + + static ScalarFunction GetFunction(); +}; + +struct ArrayReduceFun { + using ALIAS = ListReduceFun; + + static constexpr const char *Name = "array_reduce"; +}; + +struct ReduceFun { + using ALIAS = ListReduceFun; + + static constexpr const char *Name = "reduce"; +}; + struct GenerateSeriesFun { static constexpr const char *Name = "generate_series"; static constexpr const char *Parameters = "start,stop,step"; diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/secret_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/secret_functions.hpp new file mode 100644 index 000000000..17e5614e0 --- /dev/null +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/secret_functions.hpp @@ -0,0 +1,27 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/core_functions/scalar/secret_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 WhichSecretFun { + static constexpr const char *Name = "which_secret"; + static constexpr const char *Parameters = "path,type"; + static constexpr const char *Description = "Print out the name of the secret that will be used for reading a path"; + static constexpr const char *Example = "which_secret('s3://some/authenticated/path.csv', 's3')"; + + static ScalarFunction GetFunction(); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/core_functions/scalar/string_functions.hpp b/src/duckdb/src/include/duckdb/core_functions/scalar/string_functions.hpp index f3e1fa509..f9a60fcad 100644 --- a/src/duckdb/src/include/duckdb/core_functions/scalar/string_functions.hpp +++ b/src/duckdb/src/include/duckdb/core_functions/scalar/string_functions.hpp @@ -279,6 +279,42 @@ struct MD5NumberUpperFun { static ScalarFunction GetFunction(); }; +struct ParseDirnameFun { + static constexpr const char *Name = "parse_dirname"; + static constexpr const char *Parameters = "string,separator"; + static constexpr const char *Description = "Returns the top-level directory name. separator options: system, both_slash (default), forward_slash, backslash"; + static constexpr const char *Example = "parse_dirname('path/to/file.csv', 'system')"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ParseDirpathFun { + static constexpr const char *Name = "parse_dirpath"; + static constexpr const char *Parameters = "string,separator"; + static constexpr const char *Description = "Returns the head of the path similarly to Python's os.path.dirname. separator options: system, both_slash (default), forward_slash, backslash"; + static constexpr const char *Example = "parse_dirpath('path/to/file.csv', 'system')"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ParseFilenameFun { + static constexpr const char *Name = "parse_filename"; + static constexpr const char *Parameters = "string,trim_extension,separator"; + static constexpr const char *Description = "Returns the last component of the path similarly to Python's os.path.basename. If trim_extension is true, the file extension will be removed (it defaults to false). separator options: system, both_slash (default), forward_slash, backslash"; + static constexpr const char *Example = "parse_filename('path/to/file.csv', true, 'forward_slash')"; + + static ScalarFunctionSet GetFunctions(); +}; + +struct ParsePathFun { + static constexpr const char *Name = "parse_path"; + static constexpr const char *Parameters = "string,separator"; + static constexpr const char *Description = "Returns a list of the components (directories and filename) in the path similarly to Python's pathlib.PurePath::parts. separator options: system, both_slash (default), forward_slash, backslash"; + static constexpr const char *Example = "parse_path('path/to/file.csv', 'system')"; + + static ScalarFunctionSet GetFunctions(); +}; + struct PrintfFun { static constexpr const char *Name = "printf"; static constexpr const char *Parameters = "format,parameters..."; @@ -480,4 +516,13 @@ struct ToBaseFun { static ScalarFunctionSet GetFunctions(); }; +struct RegexpEscapeFun { + static constexpr const char *Name = "regexp_escape"; + static constexpr const char *Parameters = "string"; + static constexpr const char *Description = "Escapes all potentially meaningful regexp characters in the input string"; + static constexpr const char *Example = "regexp_escape('https://duckdb.org')"; + + static ScalarFunction GetFunction(); +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp b/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp index b8a4a8fa4..d05b35763 100644 --- a/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp +++ b/src/duckdb/src/include/duckdb/execution/aggregate_hashtable.hpp @@ -89,7 +89,7 @@ class GroupedAggregateHashTable : public BaseAggregateHashTable { public: //! The hash table load factor, when a resize is triggered - constexpr static float LOAD_FACTOR = 1.5; + constexpr static double LOAD_FACTOR = 1.5; //! Get the layout of this HT const TupleDataLayout &GetLayout() const; @@ -138,7 +138,7 @@ class GroupedAggregateHashTable : public BaseAggregateHashTable { //! Executes the filter(if any) and update the aggregates void Combine(GroupedAggregateHashTable &other); - void Combine(TupleDataCollection &other_data); + void Combine(TupleDataCollection &other_data, optional_ptr> progress = nullptr); //! Unpins the data blocks void UnpinData(); diff --git a/src/duckdb/src/include/duckdb/execution/executor.hpp b/src/duckdb/src/include/duckdb/execution/executor.hpp index d249a1a7f..767f44f95 100644 --- a/src/duckdb/src/include/duckdb/execution/executor.hpp +++ b/src/duckdb/src/include/duckdb/execution/executor.hpp @@ -13,8 +13,8 @@ #include "duckdb/common/mutex.hpp" #include "duckdb/common/pair.hpp" #include "duckdb/common/reference_map.hpp" -#include "duckdb/parallel/pipeline.hpp" #include "duckdb/execution/task_error_manager.hpp" +#include "duckdb/parallel/pipeline.hpp" namespace duckdb { class ClientContext; @@ -57,7 +57,7 @@ class Executor { unique_ptr FetchChunk(); //! Push a new error - void PushError(PreservedError exception); + void PushError(ErrorData exception); //! True if an error has been thrown bool HasError(); @@ -107,8 +107,8 @@ class Executor { static void ScheduleEventsInternal(ScheduleEventData &event_data); static void VerifyScheduledEvents(const ScheduleEventData &event_data); - static void VerifyScheduledEventsInternal(const idx_t i, const vector &vertices, vector &visited, - vector &recursion_stack); + static void VerifyScheduledEventsInternal(const idx_t i, const vector> &vertices, + vector &visited, vector &recursion_stack); static void SchedulePipeline(const shared_ptr &pipeline, ScheduleEventData &event_data); diff --git a/src/duckdb/src/include/duckdb/execution/index/art/art.hpp b/src/duckdb/src/include/duckdb/execution/index/art/art.hpp index cdb609755..7848746b4 100644 --- a/src/duckdb/src/include/duckdb/execution/index/art/art.hpp +++ b/src/duckdb/src/include/duckdb/execution/index/art/art.hpp @@ -33,6 +33,8 @@ struct ARTFlags { class ART : public Index { public: + // Index type name for the ART + static constexpr const char *TYPE_NAME = "ART"; //! FixedSizeAllocator count of the ART static constexpr uint8_t ALLOCATOR_COUNT = 6; @@ -51,22 +53,29 @@ class ART : public Index { //! True, if the ART owns its data bool owns_data; -public: - //! Initialize a single predicate scan on the index with the given expression and column IDs - unique_ptr InitializeScanSinglePredicate(const Transaction &transaction, const Value &value, - const ExpressionType expression_type) override; - //! Initialize a two predicate scan on the index with the given expression and column IDs - unique_ptr InitializeScanTwoPredicates(const Transaction &transaction, const Value &low_value, - const ExpressionType low_expression_type, - const Value &high_value, - const ExpressionType high_expression_type) override; + //! Try to initialize a scan on the index with the given expression and filter + unique_ptr TryInitializeScan(const Transaction &transaction, const Expression &index_expr, + const Expression &filter_expr); + //! Performs a lookup on the index, fetching up to max_count result IDs. Returns true if all row IDs were fetched, //! and false otherwise - bool Scan(const Transaction &transaction, const DataTable &table, IndexScanState &state, const idx_t max_count, - vector &result_ids) override; + bool Scan(const Transaction &transaction, const DataTable &table, IndexScanState &state, idx_t max_count, + vector &result_ids); + +public: + //! Create a index instance of this type + static unique_ptr Create(const string &name, const IndexConstraintType constraint_type, + const vector &column_ids, + const vector> &unbound_expressions, + TableIOManager &table_io_manager, AttachedDatabase &db, + const IndexStorageInfo &storage_info) { + auto art = make_uniq(name, constraint_type, column_ids, table_io_manager, unbound_expressions, db, nullptr, + storage_info); + return std::move(art); + } //! Called when data is appended to the index. The lock obtained from InitializeLock must be held - PreservedError Append(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; + ErrorData Append(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; //! Verify that data can be appended to the index without a constraint violation void VerifyAppend(DataChunk &chunk) override; //! Verify that data can be appended to the index without a constraint violation using the conflict manager @@ -76,7 +85,7 @@ class ART : public Index { //! Delete a chunk of entries from the index. The lock obtained from InitializeLock must be held void Delete(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; //! Insert a chunk of entries into the index - PreservedError Insert(IndexLock &lock, DataChunk &data, Vector &row_ids) override; + ErrorData Insert(IndexLock &lock, DataChunk &data, Vector &row_ids) override; //! Construct an ART from a vector of sorted keys bool ConstructFromSorted(idx_t count, vector &keys, Vector &row_identifiers); @@ -153,6 +162,9 @@ class ART : public Index { void Deserialize(const BlockPointer &pointer); //! Initializes the serialization of the index by combining the allocator data onto partial blocks void WritePartialBlocks(); + + string GetConstraintViolationMessage(VerifyExistenceType verify_type, idx_t failed_index, + DataChunk &input) override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/index_type.hpp b/src/duckdb/src/include/duckdb/execution/index/index_type.hpp new file mode 100644 index 000000000..2cd6a54fa --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/index_type.hpp @@ -0,0 +1,41 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/index_type.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/typedefs.hpp" +#include "duckdb/common/vector.hpp" +#include "duckdb/common/unique_ptr.hpp" +#include "duckdb/common/string.hpp" + +namespace duckdb { + +class Index; +enum class IndexConstraintType : uint8_t; +class Expression; +class TableIOManager; +class AttachedDatabase; +struct IndexStorageInfo; + +typedef unique_ptr (*index_create_function_t)(const string &name, + const IndexConstraintType index_constraint_type, + const vector &column_ids, + const vector> &unbound_expressions, + TableIOManager &table_io_manager, AttachedDatabase &db, + const IndexStorageInfo &storage_info); +//! A index "type" +class IndexType { +public: + // The name of the index type + string name; + + // Callbacks + index_create_function_t create_instance; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/index_type_set.hpp b/src/duckdb/src/include/duckdb/execution/index/index_type_set.hpp new file mode 100644 index 000000000..9afe93e86 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/index_type_set.hpp @@ -0,0 +1,29 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/index_type_set.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/index/index_type.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/case_insensitive_map.hpp" +#include "duckdb/common/string.hpp" +#include "duckdb/common/optional_ptr.hpp" + +namespace duckdb { + +class IndexTypeSet { + mutex lock; + case_insensitive_map_t functions; + +public: + IndexTypeSet(); + DUCKDB_API optional_ptr FindByName(const string &name); + DUCKDB_API void RegisterIndexType(const IndexType &index_type); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/index/unknown_index.hpp b/src/duckdb/src/include/duckdb/execution/index/unknown_index.hpp new file mode 100644 index 000000000..ac16846e0 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/index/unknown_index.hpp @@ -0,0 +1,65 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/index/unknown_index.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/index.hpp" +#include "duckdb/parser/parsed_data/create_index_info.hpp" + +namespace duckdb { + +// An unknown index is an index that has been created by an extension, which has not been loaded yet. +// It is used as a placeholder for the index until the extension is loaded, at which point the extension will replace +// all recognized unknown indexes with the correct index type. +// Calling any function on an unknown index will throw a NotImplementedException +class UnknownIndex final : public Index { +private: + CreateIndexInfo create_info; + IndexStorageInfo storage_info; + string GenerateErrorMessage() const; + +public: + UnknownIndex(const string &name, const string &index_type, IndexConstraintType index_constraint_type, + const vector &column_ids, TableIOManager &table_io_manager, + const vector> &unbound_expressions, AttachedDatabase &db, + const CreateIndexInfo &create_info, IndexStorageInfo storage_info); + + const CreateIndexInfo &GetCreateInfo() const { + return create_info; + } + const IndexStorageInfo &GetStorageInfo() const { + return storage_info; + } + const string &GetIndexType() { + return create_info.index_type; + } + +public: + bool IsUnknown() override { + return true; + } + + // Index interface (unused) + + ErrorData Append(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; + void VerifyAppend(DataChunk &chunk) override; + void VerifyAppend(DataChunk &chunk, ConflictManager &conflict_manager) override; + void CommitDrop(IndexLock &index_lock) override; + void Delete(IndexLock &lock, DataChunk &entries, Vector &row_identifiers) override; + ErrorData Insert(IndexLock &lock, DataChunk &data, Vector &row_ids) override; + IndexStorageInfo GetStorageInfo(bool get_buffers) override; + bool MergeIndexes(IndexLock &state, Index &other_index) override; + void Vacuum(IndexLock &state) override; + idx_t GetInMemorySize(IndexLock &index_lock) override; + void CheckConstraintsForChunk(DataChunk &input, ConflictManager &conflict_manager) override; + string VerifyAndToString(IndexLock &state, bool only_verify) override; + string GetConstraintViolationMessage(VerifyExistenceType verify_type, idx_t failed_index, + DataChunk &input) override; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/join_hashtable.hpp b/src/duckdb/src/include/duckdb/execution/join_hashtable.hpp index 0bca23340..b96589add 100644 --- a/src/duckdb/src/include/duckdb/execution/join_hashtable.hpp +++ b/src/duckdb/src/include/duckdb/execution/join_hashtable.hpp @@ -116,7 +116,7 @@ class JoinHashTable { public: JoinHashTable(BufferManager &buffer_manager, const vector &conditions, - vector build_types, JoinType type); + vector build_types, JoinType type, const vector &output_columns); ~JoinHashTable(); //! Add the given data to the HT @@ -165,6 +165,8 @@ class JoinHashTable { vector condition_types; //! The types of all conditions vector build_types; + //! Positions of the columns that need to output + const vector &output_columns; //! The comparison predicates vector predicates; //! Data column layout @@ -240,13 +242,12 @@ class JoinHashTable { //===--------------------------------------------------------------------===// // External Join //===--------------------------------------------------------------------===// + static constexpr const idx_t INITIAL_RADIX_BITS = 3; + struct ProbeSpillLocalAppendState { //! Local partition and append state (if partitioned) PartitionedColumnData *local_partition; PartitionedColumnDataAppendState *local_partition_append_state; - //! Local spill and append state (if not partitioned) - ColumnDataCollection *local_spill_collection; - ColumnDataAppendState *local_spill_append_state; }; //! ProbeSpill represents materialized probe-side data that could not be probed during PhysicalHashJoin::Execute //! because the HashTable did not fit in memory. The ProbeSpill is not partitioned if the remaining data can be @@ -274,32 +275,31 @@ class JoinHashTable { mutex lock; ClientContext &context; - //! Whether the probe data is partitioned - bool partitioned; //! The types of the probe DataChunks const vector &probe_types; //! The column ids vector column_ids; - //! The partitioned probe data (if partitioned) and append states + //! The partitioned probe data and append states unique_ptr global_partitions; vector> local_partitions; vector> local_partition_append_states; - //! The probe data (if not partitioned) and append states + //! The active probe data unique_ptr global_spill_collection; - vector> local_spill_collections; - vector> local_spill_append_states; }; - //! Whether we are doing an external hash join - bool external; - //! The current number of radix bits used to partition - idx_t radix_bits; - //! The max size of the HT - idx_t max_ht_size; - //! Total count - idx_t total_count; + idx_t GetRadixBits() const { + return radix_bits; + } + + idx_t GetPartitionStart() const { + return partition_start; + } + + idx_t GetPartitionEnd() const { + return partition_end; + } //! Capacity of the pointer table given the ht count //! (minimum of 1024 to prevent collision chance for small HT's) @@ -311,23 +311,32 @@ class JoinHashTable { return PointerTableCapacity(count) * sizeof(data_ptr_t); } - //! Whether we need to do an external join - bool RequiresExternalJoin(ClientConfig &config, vector> &local_hts); - //! Computes partition sizes and number of radix bits (called before scheduling partition tasks) - bool RequiresPartitioning(ClientConfig &config, vector> &local_hts); + //! Get total size of HT if all partitions would be built + idx_t GetTotalSize(vector> &local_hts, idx_t &max_partition_size, + idx_t &max_partition_count) const; + idx_t GetTotalSize(const vector &partition_sizes, const vector &partition_counts, + idx_t &max_partition_size, idx_t &max_partition_count) const; + //! Get the remaining size of the unbuilt partitions + idx_t GetRemainingSize(); + //! Sets number of radix bits according to the max ht size + void SetRepartitionRadixBits(vector> &local_hts, const idx_t max_ht_size, + const idx_t max_partition_size, const idx_t max_partition_count); //! Partition this HT - void Partition(JoinHashTable &global_ht); + void Repartition(JoinHashTable &global_ht); //! Delete blocks that belong to the current partitioned HT void Reset(); //! Build HT for the next partitioned probe round - bool PrepareExternalFinalize(); + bool PrepareExternalFinalize(const idx_t max_ht_size); //! Probe whatever we can, sink the rest into a thread-local HT unique_ptr ProbeAndSpill(DataChunk &keys, TupleDataChunkState &key_state, DataChunk &payload, ProbeSpill &probe_spill, ProbeSpillLocalAppendState &spill_state, DataChunk &spill_chunk); private: + //! The current number of radix bits used to partition + idx_t radix_bits; + //! First and last partition of the current probe round idx_t partition_start; idx_t partition_end; diff --git a/src/duckdb/src/include/duckdb/execution/merge_sort_tree.hpp b/src/duckdb/src/include/duckdb/execution/merge_sort_tree.hpp index c568b1563..b01d7087f 100644 --- a/src/duckdb/src/include/duckdb/execution/merge_sort_tree.hpp +++ b/src/duckdb/src/include/duckdb/execution/merge_sort_tree.hpp @@ -11,9 +11,11 @@ #include "duckdb/common/array.hpp" #include "duckdb/common/helper.hpp" #include "duckdb/common/pair.hpp" +#include "duckdb/common/printer.hpp" #include "duckdb/common/typedefs.hpp" #include "duckdb/common/vector.hpp" #include "duckdb/common/vector_operations/aggregate_executor.hpp" +#include namespace duckdb { @@ -44,6 +46,30 @@ namespace duckdb { // Implementation of a generic merge-sort-tree // Rewrite of the original, which was in C++17 and targeted for research, // instead of deployment. +template +struct MergeSortTraits { + using return_type = T; + static const return_type SENTINEL() { + return NumericLimits::Maximum(); + }; +}; + +template +struct MergeSortTraits> { + using return_type = std::tuple; + static return_type SENTINEL() { + return std::tuple {MergeSortTraits::SENTINEL()...}; + }; +}; + +template +struct MergeSortTraits> { + using return_type = std::pair; + static return_type SENTINEL() { + return std::pair {MergeSortTraits::SENTINEL()...}; + }; +}; + template , uint64_t F = 32, uint64_t C = 32> struct MergeSortTree { using ElementType = E; @@ -74,7 +100,7 @@ struct MergeSortTree { CMP cmp; }; - MergeSortTree() { + explicit MergeSortTree(const CMP &cmp = CMP()) : cmp(cmp) { } explicit MergeSortTree(Elements &&lowest_level, const CMP &cmp = CMP()); @@ -84,6 +110,15 @@ struct MergeSortTree { return tree.front().first[i]; } + template + void AggregateLowerBound(const idx_t lower, const idx_t upper, const idx_t needle, L aggregate) const; + + Tree tree; + CompareElements cmp; + + static constexpr auto FANOUT = F; + static constexpr auto CASCADING = C; + protected: RunElement StartGames(Games &losers, const RunElements &elements, const RunElement &sentinel) { const auto elem_nodes = elements.size(); @@ -157,12 +192,6 @@ struct MergeSortTree { return smallest; } - Tree tree; - CompareElements cmp; - - static constexpr auto FANOUT = F; - static constexpr auto CASCADING = C; - static idx_t LowestCascadingLevel() { idx_t level = 0; idx_t level_width = 1; @@ -172,6 +201,52 @@ struct MergeSortTree { } return level; } + + void Print() const { + std::ostringstream out; + const char *separator = " "; + const char *group_separator = " || "; + idx_t level_width = 1; + idx_t number_width = 0; + for (auto &level : tree) { + for (auto &e : level.first) { + if (e) { + idx_t digits = ceil(log10(fabs(e))) + (e < 0); + if (digits > number_width) { + number_width = digits; + } + } + } + } + for (auto &level : tree) { + // Print the elements themself + { + out << 'd'; + for (size_t i = 0; i < level.first.size(); ++i) { + out << ((i && i % level_width == 0) ? group_separator : separator); + out << std::setw(number_width) << level.first[i]; + } + out << std::endl; + } + // Print the pointers + if (!level.second.empty()) { + idx_t run_cnt = (level.first.size() + level_width - 1) / level_width; + idx_t cascading_idcs_cnt = run_cnt * (2 + level_width / CASCADING) * FANOUT; + for (idx_t child_nr = 0; child_nr < FANOUT; ++child_nr) { + out << " "; + for (idx_t idx = 0; idx < cascading_idcs_cnt; idx += FANOUT) { + out << ((idx && ((idx / FANOUT) % (level_width / CASCADING + 2) == 0)) ? group_separator + : separator); + out << std::setw(number_width) << level.second[idx + child_nr]; + } + out << std::endl; + } + } + level_width *= FANOUT; + } + + Printer::Print(out.str()); + } }; template @@ -181,7 +256,7 @@ MergeSortTree::MergeSortTree(Elements &&lowest_level, const CMP const auto count = lowest_level.size(); tree.emplace_back(Level(lowest_level, Offsets())); - const RunElement SENTINEL(std::numeric_limits::max(), std::numeric_limits::max()); + const RunElement SENTINEL(MergeSortTraits::SENTINEL(), MergeSortTraits::SENTINEL()); // Fan in parent levels until we are at the top // Note that we don't build the top layer as that would just be all the data. @@ -394,4 +469,162 @@ idx_t MergeSortTree::SelectNth(const SubFrames &frames, idx_t n return result; } +template +template +void MergeSortTree::AggregateLowerBound(const idx_t lower, const idx_t upper, const idx_t needle, + L aggregate) const { + + if (lower >= upper) { + return; + } + + D_ASSERT(upper <= tree[0].first.size()); + + using IdxRange = std::pair; + + // Find the entry point into the tree + IdxRange run_idx(lower, upper - 1); + idx_t level_width = 1; + idx_t level = 0; + IdxRange prev_run_idx; + IdxRange curr; + if (run_idx.first == run_idx.second) { + curr.first = curr.second = run_idx.first; + } else { + do { + prev_run_idx.second = run_idx.second; + run_idx.first /= FANOUT; + run_idx.second /= FANOUT; + level_width *= FANOUT; + ++level; + } while (run_idx.first != run_idx.second); + curr.second = prev_run_idx.second * level_width / FANOUT; + curr.first = curr.second; + } + + // Aggregate layers using the cascading indices + if (level > LowestCascadingLevel()) { + IdxRange cascading_idx; + // Find the initial cascading idcs + { + IdxRange entry; + entry.first = run_idx.first * level_width; + entry.second = std::min(entry.first + level_width, static_cast(tree[0].first.size())); + auto *level_data = tree[level].first.data(); + idx_t entry_idx = + std::lower_bound(level_data + entry.first, level_data + entry.second, needle) - level_data; + cascading_idx.first = cascading_idx.second = + (entry_idx / CASCADING + 2 * (entry.first / level_width)) * FANOUT; + + // We have to slightly shift the initial CASCADING idcs because at the top level + // we won't be exactly on a boundary + auto correction = (prev_run_idx.second - run_idx.second * FANOUT); + cascading_idx.first -= (FANOUT - correction); + cascading_idx.second += correction; + } + + // Aggregate all layers until we reach a layer without cascading indices + // For the first layer, we already checked we have cascading indices available, otherwise + // we wouldn't have even searched the entry points. Hence, we use a `do-while` instead of `while` + do { + --level; + level_width /= FANOUT; + auto *level_data = tree[level].first.data(); + auto &cascading_idcs = tree[level + 1].second; + // Left side of tree + // Handle all completely contained runs + cascading_idx.first += FANOUT - 1; + while (curr.first - lower >= level_width) { + // Search based on cascading info from previous level + const auto *search_begin = level_data + cascading_idcs[cascading_idx.first]; + const auto *search_end = level_data + cascading_idcs[cascading_idx.first + FANOUT]; + const auto run_pos = std::lower_bound(search_begin, search_end, needle) - level_data; + // Compute runBegin and pass it to our callback + const auto run_begin = curr.first - level_width; + aggregate(level, run_begin, run_pos); + // Update state for next round + curr.first -= level_width; + --cascading_idx.first; + } + // Handle the partial last run to find the cascading entry point for the next level + if (curr.first != lower) { + const auto *search_begin = level_data + cascading_idcs[cascading_idx.first]; + const auto *search_end = level_data + cascading_idcs[cascading_idx.first + FANOUT]; + auto idx = std::lower_bound(search_begin, search_end, needle) - level_data; + cascading_idx.first = (idx / CASCADING + 2 * (lower / level_width)) * FANOUT; + } + + // Right side of tree + // Handle all completely contained runs + while (upper - curr.second >= level_width) { + // Search based on cascading info from previous level + const auto *search_begin = level_data + cascading_idcs[cascading_idx.second]; + const auto *search_end = level_data + cascading_idcs[cascading_idx.second + FANOUT]; + const auto run_pos = std::lower_bound(search_begin, search_end, needle) - level_data; + // Compute runBegin and pass it to our callback + const auto run_begin = curr.second; + aggregate(level, run_begin, run_pos); + // Update state for next round + curr.second += level_width; + ++cascading_idx.second; + } + // Handle the partial last run to find the cascading entry point for the next level + if (curr.second != upper) { + const auto *search_begin = level_data + cascading_idcs[cascading_idx.second]; + const auto *search_end = level_data + cascading_idcs[cascading_idx.second + FANOUT]; + auto idx = std::lower_bound(search_begin, search_end, needle) - level_data; + cascading_idx.second = (idx / CASCADING + 2 * (upper / level_width)) * FANOUT; + } + } while (level >= LowestCascadingLevel()); + } + + // Handle lower levels which won't have cascading info + if (level) { + while (--level) { + level_width /= FANOUT; + auto *level_data = tree[level].first.data(); + // Left side + while (curr.first - lower >= level_width) { + const auto *search_end = level_data + curr.first; + const auto *search_begin = search_end - level_width; + const auto run_pos = std::lower_bound(search_begin, search_end, needle) - level_data; + const auto run_begin = search_begin - level_data; + aggregate(level, run_begin, run_pos); + curr.first -= level_width; + } + // Right side + while (upper - curr.second >= level_width) { + const auto *search_begin = level_data + curr.second; + const auto *search_end = search_begin + level_width; + const auto run_pos = std::lower_bound(search_begin, search_end, needle) - level_data; + const auto run_begin = search_begin - level_data; + aggregate(level, run_begin, run_pos); + curr.second += level_width; + } + } + } + + // The last layer + { + auto *level_data = tree[0].first.data(); + // Left side + auto lower_it = lower; + while (lower_it != curr.first) { + const auto *search_begin = level_data + lower_it; + const auto run_begin = lower_it; + const auto run_pos = run_begin + (*search_begin < needle); + aggregate(level, run_begin, run_pos); + ++lower_it; + } + // Right side + while (curr.second != upper) { + const auto *search_begin = level_data + curr.second; + const auto run_begin = curr.second; + const auto run_pos = run_begin + (*search_begin < needle); + aggregate(level, run_begin, run_pos); + ++curr.second; + } + } +} + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp index d36b17c0a..3d1d6749b 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_hash_aggregate.hpp @@ -93,6 +93,8 @@ class PhysicalHashAggregate : public PhysicalOperator { GlobalSourceState &gstate) const override; SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; + double GetProgress(ClientContext &context, GlobalSourceState &gstate) const override; + bool IsSource() const override { return true; } diff --git a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp index 367b48441..aad04b562 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/aggregate/physical_window.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/parallel/pipeline.hpp" diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/base_scanner.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/base_scanner.hpp new file mode 100644 index 000000000..f4c8bde83 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/base_scanner.hpp @@ -0,0 +1,228 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/base_scanner.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_error.hpp" + +namespace duckdb { + +class CSVFileScan; +class ScannerResult { +public: + ScannerResult(CSVStates &states, CSVStateMachine &state_machine); + + //! Adds a Value to the result + static inline void SetQuoted(ScannerResult &result, idx_t quoted_position) { + if (!result.quoted) { + result.quoted_position = quoted_position; + } + result.quoted = true; + } + //! Adds a Row to the result + static inline void SetEscaped(ScannerResult &result) { + result.escaped = true; + } + // Variable to keep information regarding quoted and escaped values + bool quoted = false; + bool escaped = false; + idx_t quoted_position = 0; + +protected: + CSVStates &states; + CSVStateMachine &state_machine; +}; + +//! This is the base of our CSV scanners. +//! Scanners differ on what they are used for, and consequently have different performance benefits. +class BaseScanner { +public: + explicit BaseScanner(shared_ptr buffer_manager, shared_ptr state_machine, + shared_ptr error_handler, shared_ptr csv_file_scan = nullptr, + CSVIterator iterator = {}); + + virtual ~BaseScanner() = default; + //! Returns true if the scanner is finished + bool FinishedFile(); + //! Resets the scanner + void Reset(); + //! Parses data into a output_chunk + virtual ScannerResult &ParseChunk(); + + //! Returns the result from the last Parse call. Shouts at you if you call it wrong + virtual ScannerResult &GetResult(); + + CSVIterator &GetIterator(); + + idx_t GetBoundaryIndex() { + return iterator.GetBoundaryIdx(); + } + + idx_t GetLinesRead() { + return lines_read; + } + + idx_t GetIteratorPosition() { + return iterator.pos.buffer_pos; + } + + CSVStateMachine &GetStateMachine(); + + shared_ptr csv_file_scan; + + //! If this scanner is being used for sniffing + bool sniffing = false; + //! The guy that handles errors + shared_ptr error_handler; + + //! Shared pointer to the state machine, this is used across multiple scanners + shared_ptr state_machine; + + //! States + CSVStates states; + +protected: + //! Boundaries of this scanner + CSVIterator iterator; + + //! Unique pointer to the buffer_handle, this is unique per scanner, since it also contains the necessary counters + //! To offload buffers to disk if necessary + unique_ptr cur_buffer_handle; + + //! Hold the current buffer ptr + char *buffer_handle_ptr = nullptr; + + //! Shared pointer to the buffer_manager, this is shared across multiple scanners + shared_ptr buffer_manager; + + //! If this scanner has been initialized + bool initialized = false; + //! How many lines were read by this scanner + idx_t lines_read = 0; + idx_t bytes_read = 0; + //! Internal Functions used to perform the parsing + //! Initializes the scanner + virtual void Initialize(); + + //! Process one chunk + template + void Process(T &result) { + idx_t to_pos; + const idx_t start_pos = iterator.pos.buffer_pos; + if (iterator.IsBoundarySet()) { + to_pos = iterator.GetEndPos(); + if (to_pos > cur_buffer_handle->actual_size) { + to_pos = cur_buffer_handle->actual_size; + } + } else { + to_pos = cur_buffer_handle->actual_size; + } + while (iterator.pos.buffer_pos < to_pos) { + state_machine->Transition(states, buffer_handle_ptr[iterator.pos.buffer_pos]); + switch (states.states[1]) { + case CSVState::INVALID: + T::InvalidState(result); + iterator.pos.buffer_pos++; + bytes_read = iterator.pos.buffer_pos - start_pos; + return; + case CSVState::RECORD_SEPARATOR: + if (states.states[0] == CSVState::RECORD_SEPARATOR || states.states[0] == CSVState::NOT_SET) { + lines_read++; + if (T::EmptyLine(result, iterator.pos.buffer_pos)) { + iterator.pos.buffer_pos++; + bytes_read = iterator.pos.buffer_pos - start_pos; + return; + } + } else if (states.states[0] != CSVState::CARRIAGE_RETURN) { + lines_read++; + if (T::AddRow(result, iterator.pos.buffer_pos)) { + iterator.pos.buffer_pos++; + bytes_read = iterator.pos.buffer_pos - start_pos; + return; + } + } + iterator.pos.buffer_pos++; + break; + case CSVState::CARRIAGE_RETURN: + lines_read++; + if (states.states[0] == CSVState::RECORD_SEPARATOR || states.states[0] == CSVState::NOT_SET) { + if (T::EmptyLine(result, iterator.pos.buffer_pos)) { + iterator.pos.buffer_pos++; + bytes_read = iterator.pos.buffer_pos - start_pos; + return; + } + } else if (states.states[0] != CSVState::CARRIAGE_RETURN) { + if (T::AddRow(result, iterator.pos.buffer_pos)) { + iterator.pos.buffer_pos++; + bytes_read = iterator.pos.buffer_pos - start_pos; + return; + } + } + iterator.pos.buffer_pos++; + break; + case CSVState::DELIMITER: + T::AddValue(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + break; + case CSVState::QUOTED: + if (states.states[0] == CSVState::UNQUOTED) { + T::SetEscaped(result); + } + T::SetQuoted(result, iterator.pos.buffer_pos); + iterator.pos.buffer_pos++; + while (state_machine->transition_array + .skip_quoted[static_cast(buffer_handle_ptr[iterator.pos.buffer_pos])] && + iterator.pos.buffer_pos < to_pos - 1) { + iterator.pos.buffer_pos++; + } + break; + case CSVState::ESCAPE: + T::SetEscaped(result); + iterator.pos.buffer_pos++; + break; + case CSVState::STANDARD: + iterator.pos.buffer_pos++; + while (state_machine->transition_array + .skip_standard[static_cast(buffer_handle_ptr[iterator.pos.buffer_pos])] && + iterator.pos.buffer_pos < to_pos - 1) { + iterator.pos.buffer_pos++; + } + break; + case CSVState::QUOTED_NEW_LINE: + T::QuotedNewLine(result); + iterator.pos.buffer_pos++; + break; + default: + iterator.pos.buffer_pos++; + break; + } + } + bytes_read = iterator.pos.buffer_pos - start_pos; + } + + //! Finalizes the process of the chunk + virtual void FinalizeChunkProcess(); + + //! Internal function for parse chunk + template + void ParseChunkInternal(T &result) { + if (!initialized) { + Initialize(); + initialized = true; + } + if (!iterator.done) { + Process(result); + } + FinalizeChunkProcess(); + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/column_count_scanner.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/column_count_scanner.hpp new file mode 100644 index 000000000..c641541df --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/column_count_scanner.hpp @@ -0,0 +1,70 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/column_count_scanner.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/string_value_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/base_scanner.hpp" + +namespace duckdb { + +class ColumnCountResult : public ScannerResult { +public: + ColumnCountResult(CSVStates &states, CSVStateMachine &state_machine); + inline idx_t &operator[](size_t index) { + return column_counts[index]; + } + + idx_t column_counts[STANDARD_VECTOR_SIZE]; + idx_t current_column_count = 0; + bool error = false; + bool last_value_always_empty = true; + idx_t result_position = 0; + + //! Adds a Value to the result + static inline void AddValue(ColumnCountResult &result, const idx_t buffer_pos); + //! Adds a Row to the result + static inline bool AddRow(ColumnCountResult &result, const idx_t buffer_pos); + //! Behavior when hitting an invalid state + static inline void InvalidState(ColumnCountResult &result); + //! Handles QuotedNewline State + static inline void QuotedNewLine(ColumnCountResult &result); + //! Handles EmptyLine states + static inline bool EmptyLine(ColumnCountResult &result, const idx_t buffer_pos); + inline void InternalAddRow(); +}; + +//! Scanner that goes over the CSV and figures out how many columns each row has. Used for dialect sniffing +class ColumnCountScanner : public BaseScanner { +public: + ColumnCountScanner(shared_ptr buffer_manager, const shared_ptr &state_machine, + shared_ptr error_handler); + + ~ColumnCountScanner() { + } + + ColumnCountResult &ParseChunk() override; + + ColumnCountResult &GetResult() override; + + unique_ptr UpgradeToStringValueScanner(); + +private: + void Initialize() override; + + void FinalizeChunkProcess() override; + + ColumnCountResult result; + + idx_t column_count; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer.hpp similarity index 73% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer.hpp index c49168e2d..565ccdce2 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_buffer.hpp +// duckdb/execution/operator/csv_scanner/csv_buffer.hpp // // //===----------------------------------------------------------------------===// @@ -9,7 +9,7 @@ #pragma once #include "duckdb/common/constants.hpp" -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" #include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/block_manager.hpp" #include "duckdb/storage/buffer/block_handle.hpp" @@ -18,22 +18,19 @@ namespace duckdb { class CSVBufferHandle { public: - CSVBufferHandle(BufferHandle handle_p, idx_t actual_size_p, const bool is_first_buffer_p, - const bool is_final_buffer_p, idx_t csv_global_state_p, idx_t start_position_p, idx_t file_idx_p) - : handle(std::move(handle_p)), actual_size(actual_size_p), is_first_buffer(is_first_buffer_p), - is_last_buffer(is_final_buffer_p), csv_global_start(csv_global_state_p), start_position(start_position_p), - file_idx(file_idx_p) {}; - CSVBufferHandle() - : actual_size(0), is_first_buffer(false), is_last_buffer(false), csv_global_start(0), start_position(0), - file_idx(0) {}; + CSVBufferHandle(BufferHandle handle_p, idx_t actual_size_p, const bool is_final_buffer_p, idx_t file_idx_p, + idx_t buffer_index_p) + : handle(std::move(handle_p)), actual_size(actual_size_p), is_last_buffer(is_final_buffer_p), + file_idx(file_idx_p), buffer_idx(buffer_index_p) {}; + CSVBufferHandle() : actual_size(0), is_last_buffer(false), file_idx(0), buffer_idx(0) {}; + ~CSVBufferHandle() { + } //! Handle created during allocation BufferHandle handle; const idx_t actual_size; - const bool is_first_buffer; const bool is_last_buffer; - const idx_t csv_global_start; - const idx_t start_position; const idx_t file_idx; + const idx_t buffer_idx; inline char *Ptr() { return char_ptr_cast(handle.Ptr()); } @@ -51,7 +48,7 @@ class CSVBuffer { //! Constructor for `Next()` Buffers CSVBuffer(CSVFileHandle &file_handle, ClientContext &context, idx_t buffer_size, idx_t global_csv_current_position, - idx_t file_number_p); + idx_t file_number_p, idx_t buffer_idx); //! Creates a new buffer with the next part of the CSV File shared_ptr Next(CSVFileHandle &file_handle, idx_t buffer_size, idx_t file_number); @@ -59,9 +56,6 @@ class CSVBuffer { //! Gets the buffer actual size idx_t GetBufferSize(); - //! Gets the start position of the buffer, only relevant for the first time it's scanned - idx_t GetStart(); - //! If this buffer is the last buffer of the CSV File bool IsCSVFileLastBuffer(); @@ -78,11 +72,13 @@ class CSVBuffer { return char_ptr_cast(handle.Ptr()); } + //! By default, we use CSV_BUFFER_SIZE to allocate each buffer + //! TODO: Should benchmarks other values static constexpr idx_t CSV_BUFFER_SIZE = 32000000; // 32MB //! In case the file has a size < 32MB, we will use this size instead //! This is to avoid mallocing a lot of memory for a small file //! And if it's a compressed file we can't use the actual size of the file - static constexpr idx_t CSV_MINIMUM_BUFFER_SIZE = 10000000; // 10MB + static constexpr idx_t CSV_MINIMUM_BUFFER_SIZE = 8000000; // 8MB //! If this is the last buffer of the CSV File bool last_buffer = false; @@ -90,11 +86,6 @@ class CSVBuffer { ClientContext &context; //! Actual size can be smaller than the buffer size in case we allocate it too optimistically. idx_t actual_buffer_size; - //! We need to check for Byte Order Mark, to define the start position of this buffer - //! https://en.wikipedia.org/wiki/Byte_order_mark#UTF-8 - idx_t start_position = 0; - //! If this is the first buffer of the CSV File - bool first_buffer = false; //! Global position from the CSV File where this buffer starts idx_t global_csv_start = 0; //! Number of the file that is in this buffer @@ -102,6 +93,8 @@ class CSVBuffer { //! If we can seek in the file or not. //! If we can't seek, this means we can't destroy the buffers bool can_seek; + //! Buffer Index, used as a batch index for insertion-order preservation + idx_t buffer_idx = 0; //! -------- Allocated Block ---------// //! Block created in allocation shared_ptr block; diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp new file mode 100644 index 000000000..78c046027 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp @@ -0,0 +1,69 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/main/client_context.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" + +namespace duckdb { +class CSVBuffer; +class CSVStateMachine; + +//! This class is used to manage the CSV buffers. Buffers are cached when used for auto detection. +//! When parsing, buffer are not cached and just returned. +//! A CSV Buffer Manager is created for each separate CSV File. +class CSVBufferManager { +public: + CSVBufferManager(ClientContext &context, const CSVReaderOptions &options, const string &file_path, + const idx_t file_idx); + //! Returns a buffer from a buffer id (starting from 0). If it's in the auto-detection then we cache new buffers + //! Otherwise we remove them from the cache if they are already there, or just return them bypassing the cache. + unique_ptr GetBuffer(const idx_t buffer_idx); + //! unique_ptr to the file handle, gets stolen after sniffing + unique_ptr file_handle; + //! Initializes the buffer manager, during it's construction/reset + void Initialize(); + + void UnpinBuffer(const idx_t cache_idx); + //! Returns the buffer size set for this CSV buffer manager + idx_t GetBufferSize(); + //! Returns the number of buffers in the cached_buffers cache + idx_t BufferCount(); + //! If this buffer manager is done. In the context of a buffer manager it means that it read all buffers at least + //! once. + bool Done(); + + string GetFilePath(); + + ClientContext &context; + idx_t skip_rows = 0; + +private: + //! Reads next buffer in reference to cached_buffers.front() + bool ReadNextAndCacheIt(); + //! The file index this Buffer Manager refers to + const idx_t file_idx; + //! The file path this Buffer Manager refers to + const string file_path; + //! The cached buffers + vector> cached_buffers; + //! The last buffer it was accessed + shared_ptr last_buffer; + idx_t global_csv_pos = 0; + //! The size of the buffer, if the csv file has a smaller size than this, we will use that instead to malloc less + idx_t buffer_size; + //! If this buffer manager is done (i.e., no more buffers to read beyond the ones that were cached + bool done = false; + idx_t bytes_read = 0; + //! Because the buffer manager can be accessed in Parallel we need a mutex. + mutex main_mutex; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_casting.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_casting.hpp new file mode 100644 index 000000000..1f413e9d8 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_casting.hpp @@ -0,0 +1,137 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/csv_casting.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" +#include "duckdb/common/vector_operations/unary_executor.hpp" +#include "duckdb/common/operator/cast_operators.hpp" +#include "duckdb/common/operator/decimal_cast_operators.hpp" + +namespace duckdb { +class CSVCast { + template + static bool TemplatedTryCastFloatingVector(const CSVReaderOptions &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, + idx_t &line_error) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + idx_t row = 0; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(input, result, &error_message)) { + line_error = row; + all_converted = false; + } else { + row++; + } + return result; + }); + return all_converted; + } + + template + static bool TemplatedTryCastDecimalVector(const CSVReaderOptions &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, uint8_t width, + uint8_t scale) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(input, result, &error_message, width, scale)) { + all_converted = false; + } + return result; + }); + return all_converted; + } + + struct TryCastDateOperator { + static bool Operation(const map> &options, string_t input, + date_t &result, string &error_message) { + return options.at(LogicalTypeId::DATE).GetValue().TryParseDate(input, result, error_message); + } + }; + + struct TryCastTimestampOperator { + static bool Operation(const map> &options, string_t input, + timestamp_t &result, string &error_message) { + return options.at(LogicalTypeId::TIMESTAMP).GetValue().TryParseTimestamp(input, result, error_message); + } + }; + + template + static bool TemplatedTryCastDateVector(const map> &options, + Vector &input_vector, Vector &result_vector, idx_t count, + string &error_message, idx_t &line_error) { + D_ASSERT(input_vector.GetType().id() == LogicalTypeId::VARCHAR); + bool all_converted = true; + idx_t cur_line = 0; + UnaryExecutor::Execute(input_vector, result_vector, count, [&](string_t input) { + T result; + if (!OP::Operation(options, input, result, error_message)) { + line_error = cur_line; + all_converted = false; + } + cur_line++; + return result; + }); + return all_converted; + } + +public: + static bool TryCastDateVector(const map> &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, idx_t &line_error) { + return TemplatedTryCastDateVector(options, input_vector, result_vector, count, + error_message, line_error); + } + static bool TryCastTimestampVector(const map> &options, + Vector &input_vector, Vector &result_vector, idx_t count, + string &error_message) { + idx_t line_error; + return TemplatedTryCastDateVector(options, input_vector, result_vector, + count, error_message, line_error); + } + static bool TryCastFloatingVectorCommaSeparated(const CSVReaderOptions &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, + const LogicalType &result_type, idx_t &line_error) { + switch (result_type.InternalType()) { + case PhysicalType::DOUBLE: + return TemplatedTryCastFloatingVector( + options, input_vector, result_vector, count, error_message, line_error); + case PhysicalType::FLOAT: + return TemplatedTryCastFloatingVector( + options, input_vector, result_vector, count, error_message, line_error); + default: + throw InternalException("Unimplemented physical type for floating"); + } + } + static bool TryCastDecimalVectorCommaSeparated(const CSVReaderOptions &options, Vector &input_vector, + Vector &result_vector, idx_t count, string &error_message, + const LogicalType &result_type) { + auto width = DecimalType::GetWidth(result_type); + auto scale = DecimalType::GetScale(result_type); + switch (result_type.InternalType()) { + case PhysicalType::INT16: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT32: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT64: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + case PhysicalType::INT128: + return TemplatedTryCastDecimalVector( + options, input_vector, result_vector, count, error_message, width, scale); + default: + throw InternalException("Unimplemented physical type for decimal"); + } + } +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_error.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_error.hpp new file mode 100644 index 000000000..f12acae28 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_error.hpp @@ -0,0 +1,104 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/csv_error.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/typedefs.hpp" +#include "duckdb/common/types/hash.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/unordered_map.hpp" +#include "duckdb/common/case_insensitive_map.hpp" +#include "duckdb/common/types/string_type.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" + +namespace duckdb { + +//! Object that holds information on how many lines each csv batch read. + +class LinesPerBoundary { +public: + LinesPerBoundary(); + LinesPerBoundary(idx_t boundary_idx, idx_t lines_in_batch); + + idx_t boundary_idx = 0; + idx_t lines_in_batch = 0; +}; + +enum CSVErrorType : uint8_t { + CAST_ERROR = 0, // If when casting a value from string to the column type fails + COLUMN_NAME_TYPE_MISMATCH = 1, // If there is a mismatch between Column Names and Types + INCORRECT_COLUMN_AMOUNT = 2, // If the CSV is missing a column + UNTERMINATED_QUOTES = 3, // If a quote is not terminated + SNIFFING = 4, // If something went wrong during sniffing and was not possible to find suitable candidates + MAXIMUM_LINE_SIZE = 5, // Maximum line size was exceeded by a line in the CSV File + NULLPADDED_QUOTED_NEW_VALUE = 6, // If the null_padding option is set and we have quoted new values in parallel + +}; + +class CSVError { +public: + CSVError(string error_message, CSVErrorType type, idx_t column_idx, vector row); + CSVError(string error_message, CSVErrorType type); + //! Produces error messages for column name -> type mismatch. + static CSVError ColumnTypesError(case_insensitive_map_t sql_types_per_column, const vector &names); + //! Produces error messages for casting errors + static CSVError CastError(const CSVReaderOptions &options, string &column_name, string &cast_error, + idx_t column_idx, vector &row); + //! Produces error for when the line size exceeds the maximum line size option + static CSVError LineSizeError(const CSVReaderOptions &options, idx_t actual_size); + //! Produces error for when the sniffer couldn't find viable options + static CSVError SniffingError(string &file_path); + //! Produces error messages for unterminated quoted values + static CSVError UnterminatedQuotesError(const CSVReaderOptions &options, string_t *vector_ptr, + idx_t vector_line_start, idx_t current_column); + //! Produces error messages for null_padding option is set and we have quoted new values in parallel + static CSVError NullPaddingFail(const CSVReaderOptions &options); + //! Produces error for incorrect (e.g., smaller and lower than the predefined) number of columns in a CSV Line + static CSVError IncorrectColumnAmountError(const CSVReaderOptions &options, string_t *vector_ptr, + idx_t vector_line_start, idx_t actual_columns); + //! Actual error message + string error_message; + //! Error Type + CSVErrorType type; + //! Column Index where error happened + idx_t column_idx; + //! Values from the row where error happened + vector row; +}; + +class CSVErrorHandler { +public: + CSVErrorHandler(bool ignore_errors = false); + //! Throws the error + void Error(LinesPerBoundary &error_info, CSVError &csv_error, bool force_error = false); + //! Throws the error + void Error(CSVError &csv_error); + //! Inserts a finished error info + void Insert(idx_t boundary_idx, idx_t rows); + vector> errors; + //! Return the 1-indexed line number + idx_t GetLine(LinesPerBoundary &error_info); + void NewMaxLineSize(idx_t scan_line_size); + idx_t GetMaxLineLength() { + return max_line_length; + } + +private: + //! If we should print the line of an error + bool PrintLineNumber(CSVError &error); + //! CSV Error Handler Mutex + mutex main_mutex; + //! Map of -> lines + unordered_map lines_per_batch_map; + idx_t max_line_length = 0; + bool ignore_errors = false; + + bool got_borked = false; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_handle.hpp similarity index 94% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_handle.hpp index c27538fe2..c7e70b008 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_file_handle.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_handle.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_file_handle.hpp +// duckdb/execution/operator/csv_scanner/csv_file_handle.hpp // // //===----------------------------------------------------------------------===// @@ -43,6 +43,7 @@ struct CSVFileHandle { FileCompressionType compression); static unique_ptr OpenFile(FileSystem &fs, Allocator &allocator, const string &path, FileCompressionType compression); + bool uncompressed = false; private: unique_ptr file_handle; diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp new file mode 100644 index 000000000..74af23fef --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp @@ -0,0 +1,70 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_error.hpp" + +namespace duckdb { +struct ReadCSVData; +//! Struct holding information over a CSV File we will scan +class CSVFileScan { +public: + //! Constructor for when a CSV File Scan is being constructed over information acquired during sniffing + //! This means the options are alreadu set, and the buffer manager is already up and runinng. + CSVFileScan(ClientContext &context, shared_ptr buffer_manager, + shared_ptr state_machine, const CSVReaderOptions &options, + const ReadCSVData &bind_data, const vector &column_ids, vector &file_schema); + //! Constructor for new CSV Files, we must initialize the buffer manager and the state machine + //! Path to this file + CSVFileScan(ClientContext &context, const string &file_path, const CSVReaderOptions &options, const idx_t file_idx, + const ReadCSVData &bind_data, const vector &column_ids, + const vector &file_schema); + + CSVFileScan(ClientContext &context, const string &file_name, CSVReaderOptions &options); + + const string &GetFileName(); + const vector &GetNames(); + const vector &GetTypes(); + void InitializeProjection(); + + //! Initialize the actual names and types to be scanned from the file + void InitializeFileNamesTypes(); + const string file_path; + //! File Index + idx_t file_idx; + //! Buffer Manager for the CSV File + shared_ptr buffer_manager; + //! State Machine for this file + shared_ptr state_machine; + //! How many bytes were read up to this point + atomic bytes_read {0}; + //! Size of this file + idx_t file_size; + //! Line Info used in error messages + shared_ptr error_handler; + //! Whether or not this is an on-disk file + bool on_disk_file = true; + + vector names; + vector types; + MultiFileReaderData reader_data; + + vector file_types; + + // Variables to handle projection pushdown + set projected_columns; + std::vector> projection_ids; + + //! Options for this CSV Reader + CSVReaderOptions options; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_option.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_option.hpp similarity index 94% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_option.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_option.hpp index 81915b097..8c13e2c9f 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_option.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_option.hpp @@ -1,15 +1,13 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_option.hpp +// duckdb/execution/operator/csv_scanner/csv_option.hpp // // //===----------------------------------------------------------------------===// #pragma once -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" #include "duckdb/function/scalar/strftime_format.hpp" namespace duckdb { @@ -17,10 +15,12 @@ namespace duckdb { enum class NewLineIdentifier : uint8_t { SINGLE = 1, // Either \r or \n CARRY_ON = 2, // \r\n - MIX = 3, // Hippie-Land, can't run it multithreaded - NOT_SET = 4 + NOT_SET = 3 }; +class Serializer; +class Deserializer; + //! Wrapper for CSV Options that can be manually set by the user //! It is important to make this difference for options that can be automatically sniffed AND manually set. template diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_reader_options.hpp similarity index 81% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_reader_options.hpp index 5818a5d5e..6aa349bf8 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_reader_options.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_reader_options.hpp @@ -1,49 +1,27 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_reader_options.hpp +// duckdb/execution/operator/csv_scanner/csv_reader_options.hpp // // //===----------------------------------------------------------------------===// #pragma once -#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_option.hpp" +#include "duckdb/execution/operator/csv_scanner/state_machine_options.hpp" #include "duckdb/common/map.hpp" #include "duckdb/function/scalar/strftime_format.hpp" #include "duckdb/common/types/value.hpp" #include "duckdb/common/case_insensitive_map.hpp" #include "duckdb/common/types.hpp" #include "duckdb/common/multi_file_reader_options.hpp" -#include "duckdb/execution/operator/scan/csv/csv_option.hpp" namespace duckdb { -enum class ParallelMode { AUTOMATIC = 0, PARALLEL = 1, SINGLE_THREADED = 2 }; - -//! Struct that holds the configuration of a CSV State Machine -//! Basically which char, quote and escape were used to generate it. -struct CSVStateMachineOptions { - CSVStateMachineOptions() {}; - CSVStateMachineOptions(char delimiter_p, char quote_p, char escape_p) - : delimiter(delimiter_p), quote(quote_p), escape(escape_p) {}; - - //! Delimiter to separate columns within each line - CSVOption delimiter = ','; - //! Quote used for columns that contain reserved characters, e.g ' - CSVOption quote = '\"'; - //! Escape character to escape quote character - CSVOption escape = '\0'; - - bool operator==(const CSVStateMachineOptions &other) const { - return delimiter == other.delimiter && quote == other.quote && escape == other.escape; - } -}; - struct DialectOptions { CSVStateMachineOptions state_machine_options; - //! New Line separator - CSVOption new_line = NewLineIdentifier::NOT_SET; //! Expected number of columns idx_t num_cols = 0; //! Whether or not the file has a header line @@ -53,8 +31,6 @@ struct DialectOptions { {LogicalTypeId::TIMESTAMP, {}}}; //! How many leading rows to skip CSVOption skip_rows = 0; - //! True start of the first CSV Buffer (After skipping empty lines, headers, notes and so on) - idx_t true_start = 0; }; struct CSVReaderOptions { @@ -96,7 +72,7 @@ struct CSVReaderOptions { vector auto_type_candidates = {LogicalType::VARCHAR, LogicalType::TIMESTAMP, LogicalType::DATE, LogicalType::TIME, LogicalType::DOUBLE, LogicalType::BIGINT, LogicalType::BOOLEAN, LogicalType::SQLNULL}; - //! In case the sniffer found a mismatch error from user define types or dialect + //! In case the sniffer found a mismatch error from user defined types or dialect string sniffer_user_mismatch_error; //===--------------------------------------------------------------------===// @@ -125,11 +101,8 @@ struct CSVReaderOptions { string decimal_separator = "."; //! Whether or not to pad rows that do not have enough columns with NULL values bool null_padding = false; - - //! If we are running the parallel version of the CSV Reader. In general, the system should always auto-detect - //! When it can't execute a parallel run before execution. However, there are (rather specific) situations where - //! setting up this manually might be important - ParallelMode parallel_mode; + //! If we should attempt to run parallel scanning over one file + bool parallel = true; //! User defined parameters for the csv function concatenated on a string string user_defined_parameters; diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_sniffer.hpp similarity index 88% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_sniffer.hpp index 45828b575..42066a433 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_sniffer.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_sniffer.hpp @@ -1,16 +1,17 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_sniffer.hpp +// duckdb/execution/operator/csv_scanner/csv_sniffer.hpp // // //===----------------------------------------------------------------------===// #pragma once -#include "duckdb/execution/operator/scan/csv/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" #include "duckdb/common/vector.hpp" -#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" +#include "duckdb/execution/operator/csv_scanner/quote_rules.hpp" +#include "duckdb/execution/operator/csv_scanner/column_count_scanner.hpp" namespace duckdb { struct DateTimestampSniffing { @@ -48,7 +49,8 @@ struct SetColumns { //! 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) { + inline bool IsCandidateUnacceptable(idx_t num_cols, bool null_padding, bool ignore_errors, + bool last_value_always_empty) { if (!IsSet() || ignore_errors) { // We can't say its unacceptable if it's not set or if we ignore errors return false; @@ -58,7 +60,7 @@ struct SetColumns { // 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) { + if (num_cols == size || num_cols == size + last_value_always_empty) { // Good Candidate return false; } @@ -87,19 +89,23 @@ class CSVSniffer { //! 5. Type Replacement: Replaces the types of the columns if the user specified them SnifferResult SniffCSV(bool force_match = false); + static NewLineIdentifier DetectNewLineDelimiter(CSVBufferManager &buffer_manager); + private: //! CSV State Machine Cache CSVStateMachineCache &state_machine_cache; //! Highest number of columns found idx_t max_columns_found = 0; //! Current Candidates being considered - vector> candidates; + vector> candidates; //! Reference to original CSV Options, it will be modified as a result of the sniffer. CSVReaderOptions &options; //! Buffer being used on sniffer shared_ptr buffer_manager; //! Information regarding columns that were set by user/query SetColumns set_columns; + shared_ptr error_handler; + shared_ptr detection_error_handler; //! Sets the result options void SetResultOptions(); @@ -114,18 +120,19 @@ class CSVSniffer { unordered_map> "e_candidates_map, unordered_map> &escape_candidates_map); //! 2. Generates the search space candidates for the state machines - void GenerateStateMachineSearchSpace(vector> &csv_state_machines, + void GenerateStateMachineSearchSpace(vector> &column_count_scanners, const vector &delimiter_candidates, const vector "erule_candidates, const unordered_map> "e_candidates_map, const unordered_map> &escape_candidates_map); //! 3. Analyzes if dialect candidate is a good candidate to be considered, if so, it adds it to the candidates - void AnalyzeDialectCandidate(unique_ptr, idx_t &rows_read, idx_t &best_consistent_rows, + void AnalyzeDialectCandidate(unique_ptr, idx_t &rows_read, idx_t &best_consistent_rows, idx_t &prev_padding_count); //! 4. Refine Candidates over remaining chunks void RefineCandidates(); + //! Checks if candidate still produces good values for the next chunk - bool RefineCandidateNextChunk(CSVStateMachine &candidate); + bool RefineCandidateNextChunk(ColumnCountScanner &candidate); //! ------------------------------------------------------// //! ------------------- Type Detection ------------------ // @@ -137,6 +144,7 @@ class CSVSniffer { //! Try to cast a string value to the specified sql type bool TryCastValue(CSVStateMachine &candidate, const Value &value, const LogicalType &sql_type); void SetDateFormat(CSVStateMachine &candidate, const string &format_specifier, const LogicalTypeId &sql_type); + //! Function that initialized the necessary variables used for date and timestamp detection void InitializeDateAndTimeStampDetection(CSVStateMachine &candidate, const string &separator, const LogicalType &sql_type); @@ -154,9 +162,7 @@ class CSVSniffer { }; unordered_map> best_sql_types_candidates_per_column_idx; map> best_format_candidates; - unique_ptr best_candidate; - idx_t best_start_with_header = 0; - idx_t best_start_without_header = 0; + unique_ptr best_candidate; vector best_header_row; //! Variable used for sniffing date and timestamp map format_candidates; diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state.hpp similarity index 69% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state.hpp index 02137eefd..d2993e037 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_state.hpp +// duckdb/execution/operator/csv_scanner/csv_state.hpp // // //===----------------------------------------------------------------------===// @@ -21,8 +21,10 @@ enum class CSVState : uint8_t { QUOTED = 4, //! State when inside a quoted field UNQUOTED = 5, //! State when leaving a quoted field ESCAPE = 6, //! State when encountering an escape character (e.g., \) - EMPTY_LINE = 7, //! State when encountering an empty line (i.e., \r\r \n\n, \n\r) - INVALID = 8 //! Got to an Invalid State, this should error. + INVALID = 7, //! Got to an Invalid State, this should error. + NOT_SET = 8, //! If the state is not set, usually the first state before getting the first character + QUOTED_NEW_LINE = 9, //! If we have a quoted newline + EMPTY_SPACE = 10 //! If we have empty spaces in the beginning and end of value }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine.hpp new file mode 100644 index 000000000..49542782f --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine.hpp @@ -0,0 +1,99 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/csv_state_machine.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp" + +namespace duckdb { + +//! State of necessary CSV States to parse file +//! Current, previous, and state before the previous +struct CSVStates { + void Initialize() { + states[0] = CSVState::NOT_SET; + states[1] = CSVState::NOT_SET; + } + inline bool NewValue() { + return states[1] == CSVState::DELIMITER; + } + + inline bool NewRow() { + // It is a new row, if the previous state is not a record separator, and the current one is + return states[0] != CSVState::RECORD_SEPARATOR && states[0] != CSVState::CARRIAGE_RETURN && + (states[1] == CSVState::RECORD_SEPARATOR || states[1] == CSVState::CARRIAGE_RETURN); + } + + inline bool EmptyLastValue() { + // It is a new row, if the previous state is not a record separator, and the current one is + return states[0] == CSVState::DELIMITER && + (states[1] == CSVState::RECORD_SEPARATOR || states[1] == CSVState::CARRIAGE_RETURN); + } + + inline bool EmptyLine() { + return (states[1] == CSVState::CARRIAGE_RETURN || states[1] == CSVState::RECORD_SEPARATOR) && + (states[0] == CSVState::RECORD_SEPARATOR || states[0] == CSVState::NOT_SET); + } + + inline bool IsNotSet() { + return states[1] == CSVState::NOT_SET; + } + + inline bool IsCurrentNewRow() { + return states[1] == CSVState::RECORD_SEPARATOR || states[1] == CSVState::CARRIAGE_RETURN; + } + + inline bool IsCarriageReturn() { + return states[1] == CSVState::CARRIAGE_RETURN; + } + + inline bool IsQuoted() { + return states[0] == CSVState::QUOTED; + } + inline bool IsEscaped() { + return states[1] == CSVState::ESCAPE || (states[0] == CSVState::UNQUOTED && states[1] == CSVState::QUOTED); + } + inline bool IsQuotedCurrent() { + return states[1] == CSVState::QUOTED || states[1] == CSVState::QUOTED_NEW_LINE; + } + CSVState states[2]; +}; + +//! The CSV State Machine comprises a state transition array (STA). +//! The STA indicates the current state of parsing based on both the current and preceding characters. +//! This reveals whether we are dealing with a Field, a New Line, a Delimiter, and so forth. +//! The STA's creation depends on the provided quote, character, and delimiter options for that state machine. +//! The motivation behind implementing an STA is to remove branching in regular CSV Parsing by predicting and detecting +//! the states. Note: The State Machine is currently utilized solely in the CSV Sniffer. +class CSVStateMachine { +public: + explicit CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options, + CSVStateMachineCache &csv_state_machine_cache_p); + + explicit CSVStateMachine(const StateMachine &transition_array, const CSVReaderOptions &options); + + //! Transition all states to next state, that depends on the current char + inline void Transition(CSVStates &states, char current_char) const { + states.states[0] = states.states[1]; + states.states[1] = transition_array[static_cast(current_char)][static_cast(states.states[1])]; + } + + //! The Transition Array is a Finite State Machine + //! It holds the transitions of all states, on all 256 possible different characters + const StateMachine &transition_array; + //! Options of this state machine + const CSVStateMachineOptions state_machine_options; + //! CSV Reader Options + const CSVReaderOptions &options; + //! Dialect options resulting from sniffing + DialectOptions dialect_options; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp similarity index 56% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp index 416749a82..d400d810b 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp @@ -1,33 +1,39 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp +// duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp // // //===----------------------------------------------------------------------===// #pragma once -#include "duckdb/execution/operator/scan/csv/csv_state.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" -#include "duckdb/execution/operator/scan/csv/quote_rules.hpp" +#include "duckdb/storage/object_cache.hpp" +#include "duckdb/common/types/hash.hpp" +#include "duckdb/execution/operator/csv_scanner/state_machine_options.hpp" +#include "duckdb/execution/operator/csv_scanner/quote_rules.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state.hpp" namespace duckdb { //! Class to wrap the state machine matrix class StateMachine { public: - static constexpr uint32_t NUM_STATES = 9; + static constexpr uint32_t NUM_STATES = 11; static constexpr uint32_t NUM_TRANSITIONS = 256; - CSVState state_machine[NUM_STATES][NUM_TRANSITIONS]; + CSVState state_machine[NUM_TRANSITIONS][NUM_STATES]; + //! Transitions where we might skip processing + //! For the Standard State + bool skip_standard[256]; + //! For the Quoted State + bool skip_quoted[256]; - const CSVState *operator[](CSVState state) const { - return state_machine[static_cast(state)]; + const CSVState *operator[](idx_t i) const { + return state_machine[i]; } - CSVState *operator[](CSVState state) { - return state_machine[static_cast(state)]; + CSVState *operator[](idx_t i) { + return state_machine[i]; } }; @@ -37,29 +43,45 @@ struct HashCSVStateMachineConfig { auto h_delimiter = Hash(config.delimiter.GetValue()); auto h_quote = Hash(config.quote.GetValue()); auto h_escape = Hash(config.escape.GetValue()); - return CombineHash(h_delimiter, CombineHash(h_quote, h_escape)); + auto h_newline = Hash((uint8_t)config.new_line.GetValue()); + return CombineHash(h_delimiter, CombineHash(h_quote, CombineHash(h_escape, h_newline))); } }; //! The CSVStateMachineCache caches state machines, although small ~2kb, the actual creation of multiple State Machines //! can become a bottleneck on sniffing, when reading very small csv files. //! Hence the cache stores State Machines based on their different delimiter|quote|escape options. -class CSVStateMachineCache { +class CSVStateMachineCache : public ObjectCacheEntry { public: CSVStateMachineCache(); - ~CSVStateMachineCache() {}; + ~CSVStateMachineCache() override = default; + //! Gets a state machine from the cache, if it's not from one the default options + //! It first caches it, then returns it. + static CSVStateMachineCache &Get(ClientContext &context); + //! Gets a state machine from the cache, if it's not from one the default options //! It first caches it, then returns it. const StateMachine &Get(const CSVStateMachineOptions &state_machine_options); + static const string STATE_KEY; + static string ObjectType() { + return "CSV_STATE_MACHINE_CACHE"; + } + + string GetObjectType() override { + return ObjectType(); + } + private: void Insert(const CSVStateMachineOptions &state_machine_options); - //! Cache on delimiter|quote|escape + //! Cache on delimiter|quote|escape|newline unordered_map state_machine_cache; //! Default value for options used to intialize CSV State Machine Cache const vector default_delimiter = {',', '|', ';', '\t'}; const vector> default_quote = {{'\"'}, {'\"', '\''}, {'\0'}}; const vector default_quote_rule = {QuoteRule::QUOTES_RFC, QuoteRule::QUOTES_OTHER, QuoteRule::NO_QUOTES}; const vector> default_escape = {{'\0', '\"', '\''}, {'\\'}, {'\0'}}; + //! Because the state machine cache can be accessed in Parallel we need a mutex. + mutex main_mutex; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/global_csv_state.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/global_csv_state.hpp new file mode 100644 index 000000000..c227c76e1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/global_csv_state.hpp @@ -0,0 +1,79 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/global_csv_state.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_error.hpp" +#include "duckdb/function/table/read_csv.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/string_value_scanner.hpp" + +namespace duckdb { + +//! CSV Global State is used in the CSV Reader Table Function, it controls what each thread +struct CSVGlobalState : public GlobalTableFunctionState { +public: + CSVGlobalState(ClientContext &context, const shared_ptr &buffer_manager_p, + const CSVReaderOptions &options, idx_t system_threads_p, const vector &files, + vector column_ids_p, const ReadCSVData &bind_data); + + ~CSVGlobalState() override { + } + + //! Generates a CSV Scanner, with information regarding the piece of buffer it should be read. + //! In case it returns a nullptr it means we are done reading these files. + unique_ptr Next(); + + void FillRejectsTable(); + + void DecrementThread(); + + //! Returns Current Progress of this CSV Read + double GetProgress(const ReadCSVData &bind_data) const; + + //! Calculates the Max Threads that will be used by this CSV Reader + idx_t MaxThreads() const override; + //! We hold information on the current scanner boundary + CSVIterator current_boundary; + +private: + //! Reference to the client context that created this scan + ClientContext &context; + + vector> file_scans; + + //! Mutex to lock when getting next batch of bytes (Parallel Only) + mutex main_mutex; + + //! Basically max number of threads in DuckDB + idx_t system_threads; + + //! Number of threads being used in this scanner + idx_t running_threads = 1; + //! The column ids to read + vector column_ids; + + string sniffer_mismatch_error; + + bool finished = false; + + const ReadCSVData &bind_data; + + vector file_schema; + + bool single_threaded = false; + + atomic scanner_idx; + + atomic last_file_idx; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/quote_rules.hpp similarity index 91% rename from src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp rename to src/duckdb/src/include/duckdb/execution/operator/csv_scanner/quote_rules.hpp index 4dc767170..4579eb11d 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/quote_rules.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/quote_rules.hpp @@ -1,7 +1,7 @@ //===----------------------------------------------------------------------===// // DuckDB // -// duckdb/execution/operator/scan/csv/quote_rules.hpp +// duckdb/execution/operator/csv_scanner/quote_rules.hpp // // //===----------------------------------------------------------------------===// diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/scanner_boundary.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/scanner_boundary.hpp new file mode 100644 index 000000000..7cf976957 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/scanner_boundary.hpp @@ -0,0 +1,93 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/scanner_boundary.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/common/fast_mem.hpp" + +//! We all need boundaries every now and then, CSV Scans also need them +//! This class keeps track of what a scan should read, so which buffer and from where to where +//! As in real life, much like my childhood country, the rules are not really enforced. +//! So the end boundaries of a Scanner Boundary can and will be pushed. +//! In practice this means that a scanner is tolerated to read one line over it's end. +namespace duckdb { + +//! Information stored in the buffer +struct CSVBoundary { + CSVBoundary(idx_t file_idx, idx_t buffer_idx, idx_t buffer_pos, idx_t boundary_idx, idx_t end_pos); + CSVBoundary(); + void Print(); + //! File index where we start scanning [0-idx], a scanner can never go over one file. + idx_t file_idx = 0; + //! Start Buffer index of the file where we start scanning + idx_t buffer_idx = 0; + //! Start Buffer position of the buffer of the file where we start scanning + //! This position moves as we move through the buffer + idx_t buffer_pos = 0; + //! The boundary index relative to the total scan, only used for parallel reading to enforce + //! Insertion Order + idx_t boundary_idx = 0; + //! Last position this iterator should read. + idx_t end_pos; +}; + +struct CSVPosition { + CSVPosition(idx_t file_idx, idx_t buffer_idx, idx_t buffer_pos); + CSVPosition(); + //! File index where we start scanning [0-idx], a scanner can never go over one file. + idx_t file_idx = 0; + //! Start Buffer index of the file where we start scanning + idx_t buffer_idx = 0; + //! Start Buffer position of the buffer of the file where we start scanning + //! This position moves as we move through the buffer + idx_t buffer_pos = 0; +}; +struct CSVIterator { +public: + CSVIterator(idx_t file_idx, idx_t buffer_idx, idx_t buffer_pos, idx_t boundary_idx, idx_t buffer_size); + + CSVIterator(); + + void Print(); + //! Moves the boundary to the next one to be scanned, if there are no next boundaries, it returns False + //! Otherwise, if there are boundaries, it returns True + bool Next(CSVBufferManager &buffer_manager); + //! If boundary is set + bool IsBoundarySet() const; + + //! Getters + idx_t GetEndPos() const; + idx_t GetFileIdx() const; + idx_t GetBufferIdx() const; + idx_t GetBoundaryIdx() const; + + void SetCurrentPositionToBoundary(); + + void SetStart(idx_t pos); + + //! 8 MB TODO: Should benchmarks other values + static constexpr idx_t BYTES_PER_THREAD = 8000000; + + CSVPosition pos; + + bool done = false; + +private: + //! The original setting + CSVBoundary boundary; + //! Sometimes life knows no boundaries. + //! The boundaries don't have to be set for single-threaded execution. + bool is_set; +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/skip_scanner.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/skip_scanner.hpp new file mode 100644 index 000000000..7fd54ee1b --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/skip_scanner.hpp @@ -0,0 +1,60 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/skip_scanner.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/string_value_scanner.hpp" +#include "duckdb/execution/operator/csv_scanner/base_scanner.hpp" + +namespace duckdb { + +class SkipResult : public ScannerResult { +public: + SkipResult(CSVStates &states, CSVStateMachine &state_machine, idx_t rows_to_skip); + + idx_t row_count = 0; + idx_t rows_to_skip; + + //! Adds a Value to the result + static inline void AddValue(SkipResult &result, const idx_t buffer_pos); + //! Adds a Row to the result + static inline bool AddRow(SkipResult &result, const idx_t buffer_pos); + //! Behavior when hitting an invalid state + static inline void InvalidState(SkipResult &result); + //! Handles EmptyLine states + static inline bool EmptyLine(SkipResult &result, const idx_t buffer_pos); + //! Handles QuotedNewline State + static inline void QuotedNewLine(SkipResult &result); + inline void InternalAddRow(); +}; + +//! Scanner used to skip lines in a CSV File +class SkipScanner : public BaseScanner { +public: + SkipScanner(shared_ptr buffer_manager, const shared_ptr &state_machine, + shared_ptr error_handler, idx_t rows_to_skip); + + ~SkipScanner() { + } + + SkipResult &ParseChunk() override; + + SkipResult &GetResult() override; + +private: + void Initialize() override; + + void FinalizeChunkProcess() override; + + SkipResult result; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/state_machine_options.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/state_machine_options.hpp new file mode 100644 index 000000000..ee2372092 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/state_machine_options.hpp @@ -0,0 +1,35 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/state_machine_options.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_option.hpp" + +namespace duckdb { +//! Struct that holds the configuration of a CSV State Machine +//! Basically which char, quote and escape were used to generate it. +struct CSVStateMachineOptions { + CSVStateMachineOptions() {}; + CSVStateMachineOptions(char delimiter_p, char quote_p, char escape_p, NewLineIdentifier new_line_p) + : delimiter(delimiter_p), quote(quote_p), escape(escape_p), new_line(new_line_p) {}; + + //! Delimiter to separate columns within each line + CSVOption delimiter = ','; + //! Quote used for columns that contain reserved characters, e.g ' + CSVOption quote = '\"'; + //! Escape character to escape quote character + CSVOption escape = '\0'; + //! New Line separator + CSVOption new_line = NewLineIdentifier::NOT_SET; + + bool operator==(const CSVStateMachineOptions &other) const { + return delimiter == other.delimiter && quote == other.quote && escape == other.escape && + new_line == other.new_line; + } +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/string_value_scanner.hpp b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/string_value_scanner.hpp new file mode 100644 index 000000000..0a8ba39c6 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/csv_scanner/string_value_scanner.hpp @@ -0,0 +1,173 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/csv_scanner/string_value_scanner.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine.hpp" +#include "duckdb/execution/operator/csv_scanner/scanner_boundary.hpp" +#include "duckdb/execution/operator/csv_scanner/base_scanner.hpp" + +namespace duckdb { + +//! Class that keeps track of line starts, used for line size verification +class LinePosition { +public: + LinePosition() { + } + LinePosition(idx_t buffer_idx_p, idx_t buffer_pos_p, idx_t buffer_size_p) + : buffer_pos(buffer_pos_p), buffer_size(buffer_size_p), buffer_idx(buffer_idx_p) { + } + + idx_t operator-(const LinePosition &other) { + if (other.buffer_idx == buffer_idx) { + return buffer_pos - other.buffer_pos; + } + return other.buffer_size - other.buffer_pos + buffer_pos; + } + idx_t buffer_pos = 0; + idx_t buffer_size = 0; + idx_t buffer_idx = 0; +}; + +class StringValueResult : public ScannerResult { +public: + StringValueResult(CSVStates &states, CSVStateMachine &state_machine, CSVBufferHandle &buffer_handle, + Allocator &buffer_allocator, idx_t result_size, idx_t buffer_position, + CSVErrorHandler &error_hander, CSVIterator &iterator, bool store_line_size, + shared_ptr csv_file_scan, idx_t &lines_read); + + //! Information on the vector + unsafe_vector vector_ptr; + unsafe_vector validity_mask; + + //! Variables to iterate over the CSV buffers + idx_t last_position; + char *buffer_ptr; + idx_t buffer_size; + + //! CSV Options that impact the parsing + const uint32_t number_of_columns; + const bool null_padding; + const bool ignore_errors; + const char *null_str_ptr; + const idx_t null_str_size; + + //! Internal Data Chunk used for flushing + DataChunk parse_chunk; + idx_t number_of_rows = 0; + idx_t cur_col_id = 0; + idx_t result_size; + //! Information to properly handle errors + CSVErrorHandler &error_handler; + CSVIterator &iterator; + //! Where the previous line started, used to validate the maximum_line_size option + LinePosition previous_line_start; + LinePosition pre_previous_line_start; + bool store_line_size = false; + bool added_last_line = false; + bool quoted_new_line = false; + + unsafe_unique_array parse_types; + vector names; + unordered_map cast_errors; + + shared_ptr csv_file_scan; + idx_t &lines_read; + //! Information regarding projected columns + unsafe_unique_array projected_columns; + bool projecting_columns = false; + idx_t chunk_col_id = 0; + //! Specialized code for quoted values, makes sure to remove quotes and escapes + static inline void AddQuotedValue(StringValueResult &result, const idx_t buffer_pos); + //! Adds a Value to the result + static inline void AddValue(StringValueResult &result, const idx_t buffer_pos); + //! Adds a Row to the result + static inline bool AddRow(StringValueResult &result, const idx_t buffer_pos); + //! Behavior when hitting an invalid state + static inline void InvalidState(StringValueResult &result); + //! Handles QuotedNewline State + static inline void QuotedNewLine(StringValueResult &result); + void NullPaddingQuotedNewlineCheck(); + //! Handles EmptyLine states + static inline bool EmptyLine(StringValueResult &result, const idx_t buffer_pos); + inline bool AddRowInternal(); + + void HandleOverLimitRows(); + + inline void AddValueToVector(const char *value_ptr, const idx_t size, bool allocate = false); + + Value GetValue(idx_t row_idx, idx_t col_idx); + + DataChunk &ToChunk(); +}; + +//! Our dialect scanner basically goes over the CSV and actually parses the values to a DuckDB vector of string_t +class StringValueScanner : public BaseScanner { +public: + StringValueScanner(idx_t scanner_idx, const shared_ptr &buffer_manager, + const shared_ptr &state_machine, + const shared_ptr &error_handler, const shared_ptr &csv_file_scan, + CSVIterator boundary = {}, idx_t result_size = STANDARD_VECTOR_SIZE); + + StringValueScanner(const shared_ptr &buffer_manager, + const shared_ptr &state_machine, + const shared_ptr &error_handler); + + ~StringValueScanner() { + } + + StringValueResult &ParseChunk() override; + + //! Flushes the result to the insert_chunk + void Flush(DataChunk &insert_chunk); + + //! Function that creates and returns a non-boundary CSV Scanner, can be used for internal csv reading. + static unique_ptr GetCSVScanner(ClientContext &context, CSVReaderOptions &options); + + bool FinishedIterator(); + + //! Creates a new string with all escaped values removed + static string_t RemoveEscape(const char *str_ptr, idx_t end, char escape, Vector &vector); + + //! If we can directly cast the type when consuming the CSV file, or we have to do it later + static bool CanDirectlyCast(const LogicalType &type, + const map> &format_options); + + const idx_t scanner_idx; + +private: + void Initialize() override; + + void FinalizeChunkProcess() override; + + //! Function used to process values that go over the first buffer, extra allocation might be necessary + void ProcessOverbufferValue(); + + void ProcessExtraRow(); + //! Function used to move from one buffer to the other, if necessary + bool MoveToNextBuffer(); + + //! BOM skipping (https://en.wikipedia.org/wiki/Byte_order_mark) + void SkipBOM(); + + //! Skips Notes, notes are dirty lines on top of the file, before the actual data + void SkipCSVRows(); + + void SkipUntilNewLine(); + + void SetStart(); + + StringValueResult result; + vector types; + + //! Pointer to the previous buffer handle, necessary for overbuffer values + unique_ptr previous_buffer_handle; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/helper/physical_create_secret.hpp b/src/duckdb/src/include/duckdb/execution/operator/helper/physical_create_secret.hpp new file mode 100644 index 000000000..0a303b2df --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/helper/physical_create_secret.hpp @@ -0,0 +1,38 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/helper/physical_pragma.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/physical_operator.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" + +namespace duckdb { + +//! PhysicalCreateSecret represents the CREATE SECRET operator +class PhysicalCreateSecret : public PhysicalOperator { +public: + static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::CREATE_SECRET; + +public: + PhysicalCreateSecret(CreateSecretInfo info_p, idx_t estimated_cardinality) + : PhysicalOperator(PhysicalOperatorType::CREATE_SECRET, {LogicalType::BOOLEAN}, estimated_cardinality), + info(std::move(info_p)) { + } + + CreateSecretInfo info; + +public: + // Source interface + SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; + + bool IsSource() const override { + return true; + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reservoir_sample.hpp b/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reservoir_sample.hpp index 40991b727..a9e189f91 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reservoir_sample.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/helper/physical_reservoir_sample.hpp @@ -13,7 +13,9 @@ namespace duckdb { -//! PhysicalReservoirSample represents a sample taken using reservoir sampling, which is a blocking sampling method +//! PhysicalReservoirSample represents a sample taken using reservoir sampling, +//! which is a blocking sampling method + class PhysicalReservoirSample : public PhysicalOperator { public: PhysicalReservoirSample(vector types, unique_ptr options, idx_t estimated_cardinality) @@ -35,7 +37,9 @@ class PhysicalReservoirSample : public PhysicalOperator { // Sink interface SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const override; unique_ptr GetGlobalSinkState(ClientContext &context) const override; - + SinkCombineResultType Combine(ExecutionContext &context, OperatorSinkCombineInput &input) const override; + SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context, + OperatorSinkFinalizeInput &input) const override; bool ParallelSink() const override { return true; } diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_blockwise_nl_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_blockwise_nl_join.hpp index 4b1881bf9..d9b821e05 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_blockwise_nl_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_blockwise_nl_join.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/operator/join/physical_join.hpp" namespace duckdb { diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_delim_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_delim_join.hpp index 0e3a77540..d217ee038 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_delim_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_delim_join.hpp @@ -8,20 +8,17 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/physical_operator.hpp" namespace duckdb { + class PhysicalHashAggregate; -//! PhysicalDelimJoin represents a join where the LHS will be duplicate eliminated and pushed into a -//! PhysicalColumnDataScan in the RHS. +//! PhysicalDelimJoin represents a join where either the LHS or RHS will be duplicate eliminated and pushed into a +//! PhysicalColumnDataScan in the other side. Implementations are PhysicalLeftDelimJoin and PhysicalRightDelimJoin class PhysicalDelimJoin : public PhysicalOperator { public: - static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::DELIM_JOIN; - -public: - PhysicalDelimJoin(vector types, unique_ptr original_join, + PhysicalDelimJoin(PhysicalOperatorType type, vector types, unique_ptr original_join, vector> delim_scans, idx_t estimated_cardinality); unique_ptr join; @@ -31,14 +28,6 @@ class PhysicalDelimJoin : public PhysicalOperator { public: vector> GetChildren() const override; -public: - unique_ptr GetGlobalSinkState(ClientContext &context) const override; - unique_ptr GetLocalSinkState(ExecutionContext &context) 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, - OperatorSinkFinalizeInput &input) const override; - bool IsSink() const override { return true; } @@ -51,10 +40,8 @@ class PhysicalDelimJoin : public PhysicalOperator { bool SinkOrderDependent() const override { return false; } - string ParamsToString() const override; -public: - void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; + string ParamsToString() const override; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_hash_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_hash_join.hpp index 3947347d5..9fc807014 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_hash_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_hash_join.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/common/value_operations/value_operations.hpp" #include "duckdb/execution/join_hashtable.hpp" #include "duckdb/execution/operator/join/perfect_hash_join_executor.hpp" @@ -35,11 +34,19 @@ class PhysicalHashJoin : public PhysicalComparisonJoin { //! Initialize HT for this operator unique_ptr InitializeHashTable(ClientContext &context) const; - vector right_projection_map; - //! The types of the keys + //! The types of the join keys vector condition_types; - //! The types of all conditions - vector build_types; + + //! The indices for getting the payload columns + vector payload_column_idxs; + //! The types of the payload columns + vector payload_types; + + //! Positions of the RHS columns that need to output + vector rhs_output_columns; + //! The types of the output + vector rhs_output_types; + //! Duplicate eliminated types; only used for delim_joins (i.e. correlated subqueries) vector delim_types; //! Used in perfect hash join @@ -64,6 +71,8 @@ class PhysicalHashJoin : public PhysicalComparisonJoin { GlobalSourceState &gstate) const override; SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, OperatorSourceInput &input) const override; + double GetProgress(ClientContext &context, GlobalSourceState &gstate) const override; + //! Becomes a source when it is an external join bool IsSource() const override { return true; diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_join.hpp index bcc25ebaa..2d92c37b0 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_join.hpp @@ -33,7 +33,8 @@ class PhysicalJoin : public CachingPhysicalOperator { bool has_null); public: - static void BuildJoinPipelines(Pipeline ¤t, MetaPipeline &confluent_pipelines, PhysicalOperator &op); + static void BuildJoinPipelines(Pipeline ¤t, MetaPipeline &confluent_pipelines, PhysicalOperator &op, + bool build_rhs = true); void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; vector> GetSources() const override; diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_left_delim_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_left_delim_join.hpp new file mode 100644 index 000000000..9c44a9a93 --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_left_delim_join.hpp @@ -0,0 +1,37 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/join/physical_left_delim_join.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/join/physical_delim_join.hpp" + +namespace duckdb { + +//! PhysicalLeftDelimJoin represents a join where the LHS will be duplicate eliminated and pushed into a +//! PhysicalColumnDataScan in the RHS. +class PhysicalLeftDelimJoin : public PhysicalDelimJoin { +public: + static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::LEFT_DELIM_JOIN; + +public: + PhysicalLeftDelimJoin(vector types, unique_ptr original_join, + vector> delim_scans, idx_t estimated_cardinality); + +public: + unique_ptr GetGlobalSinkState(ClientContext &context) const override; + unique_ptr GetLocalSinkState(ExecutionContext &context) 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, + OperatorSinkFinalizeInput &input) const override; + +public: + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_nested_loop_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_nested_loop_join.hpp index deafc782d..85fef3ff0 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_nested_loop_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_nested_loop_join.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/operator/join/physical_comparison_join.hpp" namespace duckdb { diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp index ef9f0343d..46f30d388 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_range_join.hpp @@ -45,7 +45,7 @@ class PhysicalRangeJoin : public PhysicalComparisonJoin { private: // Merge the NULLs of all non-DISTINCT predicates into the primary so they sort to the end. - idx_t MergeNulls(const vector &conditions); + idx_t MergeNulls(Vector &primary, const vector &conditions); }; class GlobalSortedTable { diff --git a/src/duckdb/src/include/duckdb/execution/operator/join/physical_right_delim_join.hpp b/src/duckdb/src/include/duckdb/execution/operator/join/physical_right_delim_join.hpp new file mode 100644 index 000000000..394cc58bf --- /dev/null +++ b/src/duckdb/src/include/duckdb/execution/operator/join/physical_right_delim_join.hpp @@ -0,0 +1,37 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/execution/operator/join/physical_right_delim_join.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/execution/operator/join/physical_delim_join.hpp" + +namespace duckdb { + +//! PhysicalRightDelimJoin represents a join where the RHS will be duplicate eliminated and pushed into a +//! PhysicalColumnDataScan in the LHS. +class PhysicalRightDelimJoin : public PhysicalDelimJoin { +public: + static constexpr const PhysicalOperatorType TYPE = PhysicalOperatorType::RIGHT_DELIM_JOIN; + +public: + PhysicalRightDelimJoin(vector types, unique_ptr original_join, + vector> delim_scans, idx_t estimated_cardinality); + +public: + unique_ptr GetGlobalSinkState(ClientContext &context) const override; + unique_ptr GetLocalSinkState(ExecutionContext &context) 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, + OperatorSinkFinalizeInput &input) const override; + +public: + void BuildPipelines(Pipeline ¤t, MetaPipeline &meta_pipeline) override; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/order/physical_order.hpp b/src/duckdb/src/include/duckdb/execution/operator/order/physical_order.hpp index cb8a17741..2a02ffd13 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/order/physical_order.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/order/physical_order.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/parallel/pipeline.hpp" #include "duckdb/planner/bound_query_node.hpp" diff --git a/src/duckdb/src/include/duckdb/execution/operator/order/physical_top_n.hpp b/src/duckdb/src/include/duckdb/execution/operator/order/physical_top_n.hpp index 87e743f42..bf5047d68 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/order/physical_top_n.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/order/physical_top_n.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/bound_query_node.hpp" diff --git a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_copy_to_file.hpp b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_copy_to_file.hpp index 55e64dca0..e9121fd4f 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_copy_to_file.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/persistent/physical_copy_to_file.hpp @@ -8,11 +8,11 @@ #pragma once -#include "duckdb/execution/physical_operator.hpp" -#include "duckdb/parser/parsed_data/copy_info.hpp" -#include "duckdb/function/copy_function.hpp" #include "duckdb/common/file_system.hpp" #include "duckdb/common/filename_pattern.hpp" +#include "duckdb/execution/physical_operator.hpp" +#include "duckdb/function/copy_function.hpp" +#include "duckdb/parser/parsed_data/copy_info.hpp" namespace duckdb { @@ -30,9 +30,11 @@ class PhysicalCopyToFile : public PhysicalOperator { string file_path; bool use_tmp_file; FilenamePattern filename_pattern; + string file_extension; bool overwrite_or_ignore; bool parallel; bool per_thread_output; + optional_idx file_size_bytes; bool partition_output; vector partition_columns; @@ -69,5 +71,8 @@ class PhysicalCopyToFile : public PhysicalOperator { } static void MoveTmpFile(ClientContext &context, const string &tmp_file_path); + +private: + unique_ptr CreateFileState(ClientContext &context, GlobalSinkState &sink) const; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp deleted file mode 100644 index 9cc1760f5..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/base_csv_reader.hpp +++ /dev/null @@ -1,119 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/base_csv_reader.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/execution/physical_operator.hpp" -#include "duckdb/parser/parsed_data/copy_info.hpp" -#include "duckdb/function/scalar/strftime_format.hpp" -#include "duckdb/common/types/chunk_collection.hpp" -#include "duckdb/common/enums/file_compression_type.hpp" -#include "duckdb/common/map.hpp" -#include "duckdb/common/queue.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" -#include "duckdb/common/multi_file_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" - -#include - -namespace duckdb { -struct CopyInfo; -struct CSVFileHandle; -struct FileHandle; -struct StrpTimeFormat; - -class FileOpener; -class FileSystem; - -enum class ParserMode : uint8_t { PARSING = 0, SNIFFING_DATATYPES = 1, PARSING_HEADER = 2 }; - -//! Buffered CSV reader is a class that reads values from a stream and parses them as a CSV file -class BaseCSVReader { -public: - BaseCSVReader(ClientContext &context, CSVReaderOptions options, - const vector &requested_types = vector()); - virtual ~BaseCSVReader(); - - ClientContext &context; - FileSystem &fs; - Allocator &allocator; - CSVReaderOptions options; - vector return_types; - vector names; - MultiFileReaderData reader_data; - - idx_t linenr = 0; - bool linenr_estimated = false; - - bool row_empty = false; - bool bom_checked = false; - - idx_t bytes_in_chunk = 0; - - DataChunk parse_chunk; - - ParserMode mode; - -public: - const string &GetFileName() { - return options.file_path; - } - const vector &GetNames() { - return names; - } - const vector &GetTypes() { - return return_types; - } - //! Get the 1-indexed global line number for the given local error line - virtual idx_t GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first = true) { - return line_error + 1; - }; - - virtual void Increment(idx_t buffer_idx) { - return; - } - - //! Initialize projection indices to select all columns - void InitializeProjection(); - - static unique_ptr OpenCSV(ClientContext &context, const CSVReaderOptions &options); - - static bool TryCastDateVector(map> &options, Vector &input_vector, - Vector &result_vector, idx_t count, string &error_message, idx_t &line_error); - - static bool TryCastTimestampVector(map> &options, Vector &input_vector, - Vector &result_vector, idx_t count, string &error_message); - -protected: - //! Initializes the parse_chunk with varchar columns and aligns info with new number of cols - void InitParseChunk(idx_t num_cols); - //! Adds a value to the current row - void AddValue(string_t str_val, idx_t &column, vector &escape_positions, bool has_quotes, - idx_t buffer_idx = 0); - //! Adds a row to the insert_chunk, returns true if the chunk is filled as a result of this row being added - bool AddRow(DataChunk &insert_chunk, idx_t &column, string &error_message, idx_t buffer_idx = 0); - //! Finalizes a chunk, parsing all values that have been added so far and adding them to the insert_chunk - bool Flush(DataChunk &insert_chunk, idx_t buffer_idx = 0, bool try_add_line = false); - - void VerifyUTF8(idx_t col_idx); - void VerifyUTF8(idx_t col_idx, idx_t row_idx, DataChunk &chunk, int64_t offset = 0); - string GetLineNumberStr(idx_t linenr, bool linenr_estimated, idx_t buffer_idx = 0); - - //! Sets the newline delimiter - void SetNewLineDelimiter(bool carry = false, bool carry_followed_by_nl = false); - - //! Verifies that the line length did not go over a pre-defined limit. - void VerifyLineLength(idx_t line_size, idx_t buffer_idx = 0); - -protected: - //! Whether or not the current row's columns have overflown return_types.size() - bool error_column_overflow = false; - //! Number of sniffed columns - only used when auto-detecting -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp deleted file mode 100644 index 0dc156771..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp +++ /dev/null @@ -1,72 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" - -namespace duckdb { -struct CopyInfo; -struct CSVFileHandle; -struct FileHandle; -struct StrpTimeFormat; - -class FileOpener; -class FileSystem; - -//! Buffered CSV reader is a class that reads values from a stream and parses them as a CSV file -class BufferedCSVReader : public BaseCSVReader { - //! Initial buffer read size; can be extended for long lines - static constexpr idx_t INITIAL_BUFFER_SIZE = 16384; - //! Larger buffer size for non disk files - static constexpr idx_t INITIAL_BUFFER_SIZE_LARGE = 10000000; // 10MB - -public: - BufferedCSVReader(ClientContext &context, CSVReaderOptions options, - const vector &requested_types = vector()); - BufferedCSVReader(ClientContext &context, string filename, CSVReaderOptions options, - const vector &requested_types = vector()); - virtual ~BufferedCSVReader() { - } - - unsafe_unique_array buffer; - idx_t buffer_size; - idx_t position; - idx_t start = 0; - - vector> cached_buffers; - - unique_ptr file_handle; - //! CSV State Machine Cache - CSVStateMachineCache state_machine_cache; - -public: - //! Extract a single DataChunk from the CSV file and stores it in insert_chunk - void ParseCSV(DataChunk &insert_chunk); - static string ColumnTypesError(case_insensitive_map_t sql_types_per_column, const vector &names); - -private: - //! Initialize Parser - void Initialize(const vector &requested_types); - //! Skips skip_rows, reads header row from input stream - void SkipRowsAndReadHeader(idx_t skip_rows, bool skip_header); - //! Resets the buffer - void ResetBuffer(); - //! Reads a new buffer from the CSV file if the current one has been exhausted - bool ReadBuffer(idx_t &start, idx_t &line_start); - //! Try to parse a single datachunk from the file. Throws an exception if anything goes wrong. - void ParseCSV(ParserMode mode); - //! Extract a single DataChunk from the CSV file and stores it in insert_chunk - bool TryParseCSV(ParserMode mode, DataChunk &insert_chunk, string &error_message); - //! Skip Empty lines for tables with over one column - void SkipEmptyLines(); -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp deleted file mode 100644 index 169ac0be9..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp +++ /dev/null @@ -1,103 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/main/client_context.hpp" -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" - -namespace duckdb { -class CSVBuffer; -class CSVStateMachine; - -//! This class is used to manage the CSV buffers. Buffers are cached when used for auto detection. -//! When parsing, buffer are not cached and just returned. -class CSVBufferManager { -public: - CSVBufferManager(ClientContext &context, unique_ptr file_handle, const CSVReaderOptions &options, - idx_t file_idx = 0); - //! Returns a buffer from a buffer id (starting from 0). If it's in the auto-detection then we cache new buffers - //! Otherwise we remove them from the cache if they are already there, or just return them bypassing the cache. - unique_ptr GetBuffer(const idx_t pos); - //! Returns the starting position of the first buffer - idx_t GetStartPos(); - //! unique_ptr to the file handle, gets stolen after sniffing - unique_ptr file_handle; - //! Initializes the buffer manager, during it's construction/reset - void Initialize(); - - void UnpinBuffer(idx_t cache_idx); - - ClientContext &context; - idx_t skip_rows = 0; - idx_t file_idx; - bool done = false; - -private: - //! Reads next buffer in reference to cached_buffers.front() - bool ReadNextAndCacheIt(); - vector> cached_buffers; - shared_ptr last_buffer; - idx_t global_csv_pos = 0; - //! The size of the buffer, if the csv file has a smaller size than this, we will use that instead to malloc less - idx_t buffer_size; - //! Starting position of first buffer - idx_t start_pos = 0; -}; - -class CSVBufferIterator { -public: - explicit CSVBufferIterator(shared_ptr buffer_manager_p) - : buffer_manager(std::move(buffer_manager_p)) { - cur_pos = buffer_manager->GetStartPos(); - }; - - //! This functions templates an operation over the CSV File - template - inline bool Process(CSVStateMachine &machine, T &result) { - - OP::Initialize(machine); - //! If current buffer is not set we try to get a new one - if (!cur_buffer_handle) { - cur_pos = 0; - if (cur_buffer_idx == 0) { - cur_pos = buffer_manager->GetStartPos(); - } - cur_buffer_handle = buffer_manager->GetBuffer(cur_buffer_idx++); - D_ASSERT(cur_buffer_handle); - } - while (cur_buffer_handle) { - char *buffer_handle_ptr = cur_buffer_handle->Ptr(); - while (cur_pos < cur_buffer_handle->actual_size) { - if (OP::Process(machine, result, buffer_handle_ptr[cur_pos], cur_pos)) { - //! Not-Done Processing the File, but the Operator is happy! - OP::Finalize(machine, result); - return false; - } - cur_pos++; - } - cur_buffer_handle = buffer_manager->GetBuffer(cur_buffer_idx++); - cur_pos = 0; - } - //! Done Processing the File - OP::Finalize(machine, result); - return true; - } - //! Returns true if the iterator is finished - bool Finished(); - //! Resets the iterator - void Reset(); - -private: - idx_t cur_pos = 0; - idx_t cur_buffer_idx = 0; - shared_ptr buffer_manager; - unique_ptr cur_buffer_handle; -}; -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp deleted file mode 100644 index 494eb0f43..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_line_info.hpp +++ /dev/null @@ -1,47 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/csv_line_info.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -namespace duckdb { -struct LineInfo { -public: - explicit LineInfo(mutex &main_mutex_p, vector> &batch_to_tuple_end_p, - vector> &tuple_start_p, vector> &tuple_end_p, string mismatch_error) - : main_mutex(main_mutex_p), batch_to_tuple_end(batch_to_tuple_end_p), tuple_start(tuple_start_p), - tuple_end(tuple_end_p), sniffer_mismatch_error(std::move(mismatch_error)) {}; - bool CanItGetLine(idx_t file_idx, idx_t batch_idx); - - //! Return the 1-indexed line number - idx_t GetLine(idx_t batch_idx, idx_t line_error = 0, idx_t file_idx = 0, idx_t cur_start = 0, bool verify = true, - bool stop_at_first = true); - //! In case an error happened we have to increment the lines read of that batch - void Increment(idx_t file_idx, idx_t batch_idx); - //! Verify if the CSV File was read correctly from [0,batch_idx] batches. - void Verify(idx_t file_idx, idx_t batch_idx, idx_t cur_first_pos); - //! Lines read per batch, > - vector> lines_read; - //! Lines read per batch, > - vector> lines_errored; - //! Set of batches that have been initialized but are not yet finished. - vector> current_batches; - //! Pointer to CSV Reader Mutex - mutex &main_mutex; - //! Pointer Batch to Tuple End - vector> &batch_to_tuple_end; - //! Pointer Batch to Tuple Start - vector> &tuple_start; - //! Pointer Batch to Tuple End - vector> &tuple_end; - //! If we already threw an exception on a previous thread. - bool done = false; - idx_t first_line = 0; - string sniffer_mismatch_error; -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp deleted file mode 100644 index 94bb525d2..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/csv_state_machine.hpp +++ /dev/null @@ -1,70 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/csv_state_machine.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" - -namespace duckdb { - -//! The CSV State Machine comprises a state transition array (STA). -//! The STA indicates the current state of parsing based on both the current and preceding characters. -//! This reveals whether we are dealing with a Field, a New Line, a Delimiter, and so forth. -//! The STA's creation depends on the provided quote, character, and delimiter options for that state machine. -//! The motivation behind implementing an STA is to remove branching in regular CSV Parsing by predicting and detecting -//! the states. Note: The State Machine is currently utilized solely in the CSV Sniffer. -class CSVStateMachine { -public: - explicit CSVStateMachine(CSVReaderOptions &options_p, const CSVStateMachineOptions &state_machine_options, - shared_ptr buffer_manager_p, - CSVStateMachineCache &csv_state_machine_cache_p); - - //! Transition all states to next state, that depends on the current char - inline void Transition(char current_char) { - pre_previous_state = previous_state; - previous_state = state; - state = transition_array[state][static_cast(current_char)]; - } - - //! Resets the state machine, so it can be used again - void Reset(); - - //! Aux Function for string UTF8 Verification - void VerifyUTF8(); - - CSVStateMachineCache &csv_state_machine_cache; - - const CSVReaderOptions &options; - CSVBufferIterator csv_buffer_iterator; - //! Stores identified start row for this file (e.g., a file can start with garbage like notes, before the header) - idx_t start_row = 0; - //! The Transition Array is a Finite State Machine - //! It holds the transitions of all states, on all 256 possible different characters - const StateMachine &transition_array; - - //! Both these variables are used for new line identifier detection - bool single_record_separator = false; - bool carry_on_separator = false; - - //! Variables Used for Sniffing - CSVState state; - CSVState previous_state; - CSVState pre_previous_state; - idx_t cur_rows; - idx_t column_count; - string value; - idx_t rows_read; - idx_t line_start_pos = 0; - - //! Dialect options resulting from sniffing - DialectOptions dialect_options; -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp deleted file mode 100644 index a3d14feff..000000000 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp +++ /dev/null @@ -1,167 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/execution/operator/scan/csv/base_csv_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" -#include "duckdb/execution/operator/scan/csv/csv_line_info.hpp" - -#include -#include - -namespace duckdb { - -struct CSVBufferRead { - CSVBufferRead(unique_ptr buffer_p, idx_t buffer_start_p, idx_t buffer_end_p, idx_t batch_index, - idx_t local_batch_index_p, optional_ptr line_info_p) - : buffer(std::move(buffer_p)), line_info(line_info_p), buffer_start(buffer_start_p), buffer_end(buffer_end_p), - batch_index(batch_index), local_batch_index(local_batch_index_p) { - D_ASSERT(buffer); - if (buffer_end > buffer->actual_size) { - buffer_end = buffer->actual_size; - } - } - - CSVBufferRead(unique_ptr buffer_p, unique_ptr nxt_buffer_p, idx_t buffer_start_p, - idx_t buffer_end_p, idx_t batch_index, idx_t local_batch_index, optional_ptr line_info_p) - : CSVBufferRead(std::move(buffer_p), buffer_start_p, buffer_end_p, batch_index, local_batch_index, - line_info_p) { - next_buffer = std::move(nxt_buffer_p); - } - - CSVBufferRead() : buffer_start(0), buffer_end(NumericLimits::Maximum()) {}; - - const char &operator[](size_t i) const { - if (i < buffer->actual_size) { - auto buffer_ptr = buffer->Ptr(); - return buffer_ptr[i]; - } - auto next_ptr = next_buffer->Ptr(); - return next_ptr[i - buffer->actual_size]; - } - - string_t GetValue(idx_t start_buffer, idx_t position_buffer, idx_t offset) { - idx_t length = position_buffer - start_buffer - offset; - // 1) It's all in the current buffer - if (start_buffer + length <= buffer->actual_size) { - auto buffer_ptr = buffer->Ptr(); - return string_t(buffer_ptr + start_buffer, length); - } else if (start_buffer >= buffer->actual_size) { - // 2) It's all in the next buffer - D_ASSERT(next_buffer); - D_ASSERT(next_buffer->actual_size >= length + (start_buffer - buffer->actual_size)); - auto buffer_ptr = next_buffer->Ptr(); - return string_t(buffer_ptr + (start_buffer - buffer->actual_size), length); - } else { - // 3) It starts in the current buffer and ends in the next buffer - D_ASSERT(next_buffer); - auto intersection = make_unsafe_uniq_array(length); - idx_t cur_pos = 0; - auto buffer_ptr = buffer->Ptr(); - for (idx_t i = start_buffer; i < buffer->actual_size; i++) { - intersection[cur_pos++] = buffer_ptr[i]; - } - idx_t nxt_buffer_pos = 0; - auto next_buffer_ptr = next_buffer->Ptr(); - for (; cur_pos < length; cur_pos++) { - intersection[cur_pos] = next_buffer_ptr[nxt_buffer_pos++]; - } - intersections.emplace_back(std::move(intersection)); - return string_t(intersections.back().get(), length); - } - } - - unique_ptr buffer; - unique_ptr next_buffer; - vector> intersections; - optional_ptr line_info; - - idx_t buffer_start; - idx_t buffer_end; - idx_t batch_index; - idx_t local_batch_index; - idx_t lines_read = 0; -}; - -struct VerificationPositions { - idx_t beginning_of_first_line = 0; - idx_t end_of_last_line = 0; -}; - -//! CSV Reader for Parallel Reading -class ParallelCSVReader : public BaseCSVReader { -public: - ParallelCSVReader(ClientContext &context, CSVReaderOptions options, unique_ptr buffer, - idx_t first_pos_first_buffer, const vector &requested_types, idx_t file_idx_p); - virtual ~ParallelCSVReader() { - } - - //! Current Position (Relative to the Buffer) - idx_t position_buffer = 0; - - //! Start of the piece of the buffer this thread should read - idx_t start_buffer = 0; - //! End of the piece of this buffer this thread should read - idx_t end_buffer = NumericLimits::Maximum(); - //! The actual buffer size - idx_t buffer_size = 0; - - //! If this flag is set, it means we are about to try to read our last row. - bool reached_remainder_state = false; - - bool finished = false; - - unique_ptr buffer; - - idx_t file_idx; - - VerificationPositions GetVerificationPositions(); - - //! Position of the first read line and last read line for verification purposes - VerificationPositions verification_positions; - -public: - void SetBufferRead(unique_ptr buffer); - //! Extract a single DataChunk from the CSV file and stores it in insert_chunk - void ParseCSV(DataChunk &insert_chunk); - - idx_t GetLineError(idx_t line_error, idx_t buffer_idx, bool stop_at_first = true) override; - void Increment(idx_t buffer_idx) override; - -private: - //! Initialize Parser - void Initialize(const vector &requested_types); - //! Try to parse a single datachunk from the file. Throws an exception if anything goes wrong. - void ParseCSV(ParserMode mode); - //! Try to parse a single datachunk from the file. Returns whether or not the parsing is successful - bool TryParseCSV(ParserMode mode); - //! Extract a single DataChunk from the CSV file and stores it in insert_chunk - bool TryParseCSV(ParserMode mode, DataChunk &insert_chunk, string &error_message); - //! Sets Position depending on the byte_start of this thread - bool SetPosition(); - //! Called when scanning the 1st buffer, skips empty lines - bool SkipEmptyLines(); - //! When a buffer finishes reading its piece, it still can try to scan up to the real end of the buffer - //! Up to finding a new line. This function sets the buffer_end and marks a boolean variable - //! when changing the buffer end the first time. - //! It returns FALSE if the parser should jump to the final state of parsing or not - bool BufferRemainder(); - - bool NewLineDelimiter(bool carry, bool carry_followed_by_nl, bool first_char); - - //! Parses a CSV file with a one-byte delimiter, escape and quote character - bool TryParseSimpleCSV(DataChunk &insert_chunk, string &error_message, bool try_add_line = false); - - //! First Position of First Buffer - idx_t first_pos_first_buffer = 0; -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp index 17dc8edb4..57e452fb5 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/physical_column_data_scan.hpp @@ -23,10 +23,7 @@ class PhysicalColumnDataScan : public PhysicalOperator { unique_ptr owned_collection = nullptr); PhysicalColumnDataScan(vector types, PhysicalOperatorType op_type, idx_t estimated_cardinality, - idx_t cte_index) - : PhysicalOperator(op_type, std::move(types), estimated_cardinality), collection(nullptr), - cte_index(cte_index) { - } + idx_t cte_index); // the column data collection to scan optional_ptr collection; diff --git a/src/duckdb/src/include/duckdb/execution/operator/scan/physical_expression_scan.hpp b/src/duckdb/src/include/duckdb/execution/operator/scan/physical_expression_scan.hpp index 1588d5244..a6f0d1ae7 100644 --- a/src/duckdb/src/include/duckdb/execution/operator/scan/physical_expression_scan.hpp +++ b/src/duckdb/src/include/duckdb/execution/operator/scan/physical_expression_scan.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/planner/expression.hpp" diff --git a/src/duckdb/src/include/duckdb/execution/physical_operator.hpp b/src/duckdb/src/include/duckdb/execution/physical_operator.hpp index 076e5f5e7..f5876c5e7 100644 --- a/src/duckdb/src/include/duckdb/execution/physical_operator.hpp +++ b/src/duckdb/src/include/duckdb/execution/physical_operator.hpp @@ -133,7 +133,8 @@ class PhysicalOperator { // Sink interface //! The sink method is called constantly with new input, as long as new input is available. Note that this method - //! CAN be called in parallel, proper locking is needed when accessing data inside the GlobalSinkState. + //! CAN be called in parallel, proper locking is needed when accessing dat + //! a inside the GlobalSinkState. virtual SinkResultType Sink(ExecutionContext &context, DataChunk &chunk, OperatorSinkInput &input) const; // The combine is called when a single thread has completed execution of its part of the pipeline, it is the final // time that a specific LocalSinkState is accessible. This method can be called in parallel while other Sink() or diff --git a/src/duckdb/src/include/duckdb/execution/physical_plan_generator.hpp b/src/duckdb/src/include/duckdb/execution/physical_plan_generator.hpp index af19730ec..6d377a62b 100644 --- a/src/duckdb/src/include/duckdb/execution/physical_plan_generator.hpp +++ b/src/duckdb/src/include/duckdb/execution/physical_plan_generator.hpp @@ -59,6 +59,7 @@ class PhysicalPlanGenerator { unique_ptr CreatePlan(LogicalCreate &op); unique_ptr CreatePlan(LogicalCreateTable &op); unique_ptr CreatePlan(LogicalCreateIndex &op); + unique_ptr CreatePlan(LogicalCreateSecret &op); unique_ptr CreatePlan(LogicalCrossProduct &op); unique_ptr CreatePlan(LogicalDelete &op); unique_ptr CreatePlan(LogicalDelimGet &op); @@ -87,7 +88,6 @@ class PhysicalPlanGenerator { unique_ptr CreatePlan(LogicalSample &op); unique_ptr CreatePlan(LogicalSet &op); unique_ptr CreatePlan(LogicalReset &op); - unique_ptr CreatePlan(LogicalShow &op); unique_ptr CreatePlan(LogicalSimple &op); unique_ptr CreatePlan(LogicalUnnest &op); unique_ptr CreatePlan(LogicalRecursiveCTE &op); diff --git a/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp b/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp index c9827a357..2134de075 100644 --- a/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp +++ b/src/duckdb/src/include/duckdb/execution/radix_partitioned_hashtable.hpp @@ -50,8 +50,10 @@ class RadixPartitionedHashTable { SourceResultType GetData(ExecutionContext &context, DataChunk &chunk, GlobalSinkState &sink, OperatorSourceInput &input) const; + double GetProgress(ClientContext &context, GlobalSinkState &sink_p, GlobalSourceState &gstate) const; + const TupleDataLayout &GetLayout() const; - idx_t NumberOfPartitions(GlobalSinkState &sink) const; + idx_t MaxThreads(GlobalSinkState &sink) const; static void SetMultiScan(GlobalSinkState &sink); private: diff --git a/src/duckdb/src/include/duckdb/execution/reservoir_sample.hpp b/src/duckdb/src/include/duckdb/execution/reservoir_sample.hpp index 7af05d5ca..87ab341fd 100644 --- a/src/duckdb/src/include/duckdb/execution/reservoir_sample.hpp +++ b/src/duckdb/src/include/duckdb/execution/reservoir_sample.hpp @@ -8,9 +8,11 @@ #pragma once +#include "duckdb/common/allocator.hpp" #include "duckdb/common/common.hpp" #include "duckdb/common/random_engine.hpp" -#include "duckdb/common/types/chunk_collection.hpp" +#include "duckdb/common/types/data_chunk.hpp" + #include "duckdb/common/queue.hpp" namespace duckdb { @@ -24,20 +26,23 @@ class BaseReservoirSampling { void SetNextEntry(); - void ReplaceElement(); - + void ReplaceElement(double with_weight = -1); //! The random generator RandomEngine random; //! Priority queue of [random element, index] for each of the elements in the sample std::priority_queue> reservoir_weights; //! The next element to sample - idx_t next_index; + idx_t next_index_to_sample; //! The reservoir threshold of the current min entry - double min_threshold; + double min_weight_threshold; //! The reservoir index of the current min entry - idx_t min_entry; + idx_t min_weighted_entry_index; //! The current count towards next index (i.e. we will replace an entry in next_index - current_count tuples) - idx_t current_count; + //! The number of entries "seen" before choosing one that will go in our reservoir sample. + idx_t num_entries_to_skip_b4_next_sample; + //! when collecting a sample in parallel, we want to know how many values each thread has seen + //! so we can collect the samples from the thread local states in a uniform manner + idx_t num_entries_seen_total; }; class BlockingSample { @@ -50,13 +55,17 @@ class BlockingSample { //! Add a chunk of data to the sample virtual void AddToReservoir(DataChunk &input) = 0; + virtual void Finalize() = 0; //! Fetches a chunk from the sample. Note that this method is destructive and should only be used after the - // sample is completely built. + //! sample is completely built. virtual unique_ptr GetChunk() = 0; + BaseReservoirSampling base_reservoir_sample; + + virtual void Serialize(Serializer &serializer) const; + static unique_ptr Deserialize(Deserializer &deserializer); protected: //! The reservoir sampling - BaseReservoirSampling base_reservoir_sample; RandomEngine &random; }; @@ -71,19 +80,24 @@ class ReservoirSample : public BlockingSample { //! Fetches a chunk from the sample. Note that this method is destructive and should only be used after the //! sample is completely built. unique_ptr GetChunk() override; + void Finalize() override; private: //! Replace a single element of the input - void ReplaceElement(DataChunk &input, idx_t index_in_chunk); - + void ReplaceElement(DataChunk &input, idx_t index_in_chunk, double with_weight = -1); + void InitializeReservoir(DataChunk &input); //! Fills the reservoir up until sample_count entries, returns how many entries are still required idx_t FillReservoir(DataChunk &input); -private: - //! The size of the reservoir sample +public: + Allocator &allocator; + //! The size of the reservoir sample. + //! when calculating percentages, it is set to reservoir_threshold * percentage + //! when explicit number used, sample_count = number idx_t sample_count; + bool reservoir_initialized; //! The current reservoir - ChunkCollection reservoir; + unique_ptr reservoir_chunk; }; //! The reservoir sample sample_size class maintains a streaming sample of variable size @@ -99,9 +113,7 @@ class ReservoirSamplePercentage : public BlockingSample { //! Fetches a chunk from the sample. Note that this method is destructive and should only be used after the //! sample is completely built. unique_ptr GetChunk() override; - -private: - void Finalize(); + void Finalize() override; private: Allocator &allocator; @@ -109,11 +121,13 @@ class ReservoirSamplePercentage : public BlockingSample { double sample_percentage; //! The fixed sample size of the sub-reservoirs idx_t reservoir_sample_size; + //! The current sample unique_ptr current_sample; + //! The set of finished samples of the reservoir sample vector> finished_samples; - //! The amount of tuples that have been processed so far + //! The amount of tuples that have been processed so far (not put in the reservoir, just processed) idx_t current_count = 0; //! Whether or not the stream is finalized. The stream is automatically finalized on the first call to GetChunk(); bool is_finalized; diff --git a/src/duckdb/src/include/duckdb/execution/task_error_manager.hpp b/src/duckdb/src/include/duckdb/execution/task_error_manager.hpp index e50682fbd..cd47b8425 100644 --- a/src/duckdb/src/include/duckdb/execution/task_error_manager.hpp +++ b/src/duckdb/src/include/duckdb/execution/task_error_manager.hpp @@ -8,7 +8,7 @@ #pragma once -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/mutex.hpp" #include "duckdb/common/vector.hpp" @@ -16,7 +16,7 @@ namespace duckdb { class TaskErrorManager { public: - void PushError(PreservedError error) { + void PushError(ErrorData error) { lock_guard elock(error_lock); this->exceptions.push_back(std::move(error)); } @@ -41,6 +41,6 @@ class TaskErrorManager { private: mutex error_lock; //! Exceptions that occurred during the execution of the current query - vector exceptions; + vector exceptions; }; } // 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 f7f151379..73a63527f 100644 --- a/src/duckdb/src/include/duckdb/execution/window_executor.hpp +++ b/src/duckdb/src/include/duckdb/execution/window_executor.hpp @@ -176,6 +176,7 @@ class WindowAggregateExecutor : public WindowExecutor { public: bool IsConstantAggregate(); bool IsCustomAggregate(); + bool IsDistinctAggregate(); WindowAggregateExecutor(BoundWindowExpression &wexpr, ClientContext &context, const idx_t payload_count, const ValidityMask &partition_mask, const ValidityMask &order_mask, 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 28609e193..4e71f0fe1 100644 --- a/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp +++ b/src/duckdb/src/include/duckdb/execution/window_segment_tree.hpp @@ -8,6 +8,7 @@ #pragma once +#include "duckdb/common/sort/sort.hpp" #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/function/aggregate_function.hpp" @@ -48,6 +49,9 @@ class WindowAggregator { const DataChunk &GetInputs() const { return inputs; } + const ValidityMask &GetFilterMask() const { + return filter_mask; + } // Build virtual void Sink(DataChunk &payload_chunk, SelectionVector *filter_sel, idx_t filtered); @@ -58,15 +62,16 @@ class WindowAggregator { virtual void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, idx_t row_idx) const = 0; -protected: - AggregateObject aggr; + //! A description of the aggregator + const AggregateObject aggr; //! The result type of the window function - LogicalType result_type; - + const LogicalType result_type; //! The cardinality of the partition const idx_t partition_count; //! The size of a single aggregate state const idx_t state_size; + +protected: //! Partition data chunk DataChunk inputs; @@ -82,6 +87,18 @@ class WindowAggregator { const WindowExcludeMode exclude_mode; }; +// Used for validation +class WindowNaiveAggregator : public WindowAggregator { +public: + WindowNaiveAggregator(AggregateObject aggr, const LogicalType &result_type_p, + const WindowExcludeMode exclude_mode_p, idx_t partition_count); + ~WindowNaiveAggregator() override; + + unique_ptr GetLocalState() const override; + void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const override; +}; + class WindowConstantAggregator : public WindowAggregator { public: WindowConstantAggregator(AggregateObject aggr, const LogicalType &result_type_p, const ValidityMask &partition_mask, @@ -171,4 +188,47 @@ class WindowSegmentTree : public WindowAggregator { static constexpr idx_t TREE_FANOUT = 16; }; +class WindowDistinctAggregator : public WindowAggregator { +public: + using GlobalSortStatePtr = unique_ptr; + class DistinctSortTree; + + WindowDistinctAggregator(AggregateObject aggr, const LogicalType &result_type, + const WindowExcludeMode exclude_mode_p, idx_t count, ClientContext &context); + ~WindowDistinctAggregator() override; + + // Build + void Sink(DataChunk &args_chunk, SelectionVector *filter_sel, idx_t filtered) override; + void Finalize(const FrameStats &stats) override; + + // Evaluate + unique_ptr GetLocalState() const override; + void Evaluate(WindowAggregatorState &lstate, const DataChunk &bounds, Vector &result, idx_t count, + idx_t row_idx) const override; + + ClientContext &context; + ArenaAllocator allocator; + + // Single threaded sorting for now + GlobalSortStatePtr global_sort; + LocalSortState local_sort; + idx_t payload_pos; + idx_t memory_per_thread; + + vector payload_types; + DataChunk sort_chunk; + DataChunk payload_chunk; + + //! The merge sort tree for the aggregate. + unique_ptr merge_sort_tree; + + //! The actual window segment tree: an array of aggregate states that represent all the intermediate nodes + unsafe_unique_array levels_flat_native; + //! For each level, the starting location in the levels_flat_native array + vector levels_flat_start; + + //! The total number of internal nodes of the tree, stored in levels_flat_native + idx_t internal_nodes; +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/aggregate_function.hpp b/src/duckdb/src/include/duckdb/function/aggregate_function.hpp index aff5a71bf..d113b5113 100644 --- a/src/duckdb/src/include/duckdb/function/aggregate_function.hpp +++ b/src/duckdb/src/include/duckdb/function/aggregate_function.hpp @@ -89,7 +89,7 @@ class AggregateFunction : public BaseScalarFunction { aggregate_destructor_t destructor = nullptr, aggregate_statistics_t statistics = nullptr, aggregate_window_t window = nullptr, aggregate_serialize_t serialize = nullptr, aggregate_deserialize_t deserialize = nullptr) - : BaseScalarFunction(name, arguments, return_type, FunctionSideEffects::NO_SIDE_EFFECTS, + : BaseScalarFunction(name, arguments, return_type, FunctionStability::CONSISTENT, LogicalType(LogicalTypeId::INVALID), null_handling), state_size(state_size), initialize(initialize), update(update), combine(combine), finalize(finalize), simple_update(simple_update), window(window), bind(bind), destructor(destructor), statistics(statistics), @@ -103,7 +103,7 @@ class AggregateFunction : public BaseScalarFunction { aggregate_destructor_t destructor = nullptr, aggregate_statistics_t statistics = nullptr, aggregate_window_t window = nullptr, aggregate_serialize_t serialize = nullptr, aggregate_deserialize_t deserialize = nullptr) - : BaseScalarFunction(name, arguments, return_type, FunctionSideEffects::NO_SIDE_EFFECTS, + : BaseScalarFunction(name, arguments, return_type, FunctionStability::CONSISTENT, LogicalType(LogicalTypeId::INVALID)), state_size(state_size), initialize(initialize), update(update), combine(combine), finalize(finalize), simple_update(simple_update), window(window), bind(bind), destructor(destructor), statistics(statistics), diff --git a/src/duckdb/src/include/duckdb/function/aggregate_state.hpp b/src/duckdb/src/include/duckdb/function/aggregate_state.hpp index 66b7338c4..9b0015d2d 100644 --- a/src/duckdb/src/include/duckdb/function/aggregate_state.hpp +++ b/src/duckdb/src/include/duckdb/function/aggregate_state.hpp @@ -17,15 +17,19 @@ namespace duckdb { enum class AggregateType : uint8_t { NON_DISTINCT = 1, DISTINCT = 2 }; //! Whether or not the input order influences the result of the aggregate enum class AggregateOrderDependent : uint8_t { ORDER_DEPENDENT = 1, NOT_ORDER_DEPENDENT = 2 }; +//! Whether or not the combiner needs to preserve the source +enum class AggregateCombineType : uint8_t { PRESERVE_INPUT = 1, ALLOW_DESTRUCTIVE = 2 }; class BoundAggregateExpression; struct AggregateInputData { - AggregateInputData(optional_ptr bind_data_p, ArenaAllocator &allocator_p) - : bind_data(bind_data_p), allocator(allocator_p) { + AggregateInputData(optional_ptr bind_data_p, ArenaAllocator &allocator_p, + AggregateCombineType combine_type_p = AggregateCombineType::PRESERVE_INPUT) + : bind_data(bind_data_p), allocator(allocator_p), combine_type(combine_type_p) { } optional_ptr bind_data; ArenaAllocator &allocator; + AggregateCombineType combine_type; }; struct AggregateUnaryInput { diff --git a/src/duckdb/src/include/duckdb/function/cast/cast_function_set.hpp b/src/duckdb/src/include/duckdb/function/cast/cast_function_set.hpp index 3395725ce..1c6f38cea 100644 --- a/src/duckdb/src/include/duckdb/function/cast/cast_function_set.hpp +++ b/src/duckdb/src/include/duckdb/function/cast/cast_function_set.hpp @@ -13,6 +13,7 @@ namespace duckdb { struct MapCastInfo; struct MapCastNode; +struct DBConfig; typedef BoundCastInfo (*bind_cast_function_t)(BindCastInput &input, const LogicalType &source, const LogicalType &target); @@ -38,6 +39,7 @@ struct BindCastFunction { class CastFunctionSet { public: CastFunctionSet(); + CastFunctionSet(DBConfig &config); public: DUCKDB_API static CastFunctionSet &Get(ClientContext &context); @@ -57,6 +59,7 @@ class CastFunctionSet { bind_cast_function_t bind, int64_t implicit_cast_cost = -1); private: + optional_ptr config; vector bind_functions; //! If any custom cast functions have been defined using RegisterCastFunction, this holds the map optional_ptr map_info; 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 05911da6c..7835d9d43 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 @@ -214,7 +214,7 @@ struct VectorStringToArray { struct VectorStringToStruct { static bool SplitStruct(const string_t &input, vector> &varchar_vectors, idx_t &row_idx, - string_map_t &child_names, vector &child_masks); + string_map_t &child_names, vector> &child_masks); static bool StringToNestedTypeCastLoop(const string_t *source_data, ValidityMask &source_mask, Vector &result, ValidityMask &result_mask, idx_t count, CastParameters ¶meters, const SelectionVector *sel); diff --git a/src/duckdb/src/include/duckdb/function/compression/compression.hpp b/src/duckdb/src/include/duckdb/function/compression/compression.hpp index 6339ed9f2..bfc2a90d6 100644 --- a/src/duckdb/src/include/duckdb/function/compression/compression.hpp +++ b/src/duckdb/src/include/duckdb/function/compression/compression.hpp @@ -48,6 +48,16 @@ struct PatasCompressionFun { static bool TypeIsSupported(PhysicalType type); }; +struct AlpCompressionFun { + static CompressionFunction GetFunction(PhysicalType type); + static bool TypeIsSupported(PhysicalType type); +}; + +struct AlpRDCompressionFun { + static CompressionFunction GetFunction(PhysicalType type); + static bool TypeIsSupported(PhysicalType type); +}; + struct FSSTFun { static CompressionFunction GetFunction(PhysicalType type); static bool TypeIsSupported(PhysicalType type); diff --git a/src/duckdb/src/include/duckdb/function/copy_function.hpp b/src/duckdb/src/include/duckdb/function/copy_function.hpp index 8f0e5f758..2375489a0 100644 --- a/src/duckdb/src/include/duckdb/function/copy_function.hpp +++ b/src/duckdb/src/include/duckdb/function/copy_function.hpp @@ -68,10 +68,19 @@ struct PreparedBatchData { } }; +struct CopyFunctionBindInput { + const CopyInfo &info; + + string file_extension; + + CopyFunctionBindInput(const CopyInfo &info_p) : info(info_p) { + } +}; + enum class CopyFunctionExecutionMode { REGULAR_COPY_TO_FILE, PARALLEL_COPY_TO_FILE, BATCH_COPY_TO_FILE }; typedef BoundStatement (*copy_to_plan_t)(Binder &binder, CopyStatement &stmt); -typedef unique_ptr (*copy_to_bind_t)(ClientContext &context, const CopyInfo &info, +typedef unique_ptr (*copy_to_bind_t)(ClientContext &context, CopyFunctionBindInput &input, const vector &names, const vector &sql_types); typedef unique_ptr (*copy_to_initialize_local_t)(ExecutionContext &context, FunctionData &bind_data); typedef unique_ptr (*copy_to_initialize_global_t)(ClientContext &context, FunctionData &bind_data, @@ -99,18 +108,20 @@ typedef void (*copy_flush_batch_t)(ClientContext &context, FunctionData &bind_da PreparedBatchData &batch); typedef idx_t (*copy_desired_batch_size_t)(ClientContext &context, FunctionData &bind_data); +typedef idx_t (*copy_file_size_bytes_t)(GlobalFunctionData &gstate); + enum class CopyTypeSupport { SUPPORTED, LOSSY, UNSUPPORTED }; typedef CopyTypeSupport (*copy_supports_type_t)(const LogicalType &type); class CopyFunction : public Function { public: - explicit CopyFunction(string name) + explicit CopyFunction(const string &name) : Function(name), plan(nullptr), copy_to_bind(nullptr), copy_to_initialize_local(nullptr), copy_to_initialize_global(nullptr), copy_to_sink(nullptr), copy_to_combine(nullptr), copy_to_finalize(nullptr), execution_mode(nullptr), prepare_batch(nullptr), flush_batch(nullptr), - desired_batch_size(nullptr), serialize(nullptr), deserialize(nullptr), supports_type(nullptr), - copy_from_bind(nullptr) { + desired_batch_size(nullptr), file_size_bytes(nullptr), serialize(nullptr), deserialize(nullptr), + supports_type(nullptr), copy_from_bind(nullptr) { } //! Plan rewrite copy function @@ -127,6 +138,7 @@ class CopyFunction : public Function { copy_prepare_batch_t prepare_batch; copy_flush_batch_t flush_batch; copy_desired_batch_size_t desired_batch_size; + copy_file_size_bytes_t file_size_bytes; copy_to_serialize_t serialize; copy_to_deserialize_t deserialize; diff --git a/src/duckdb/src/include/duckdb/function/function.hpp b/src/duckdb/src/include/duckdb/function/function.hpp index 9a9bb9e44..7cf8e8880 100644 --- a/src/duckdb/src/include/duckdb/function/function.hpp +++ b/src/duckdb/src/include/duckdb/function/function.hpp @@ -37,7 +37,12 @@ struct PragmaInfo; //! The default null handling is NULL in, NULL out enum class FunctionNullHandling : uint8_t { DEFAULT_NULL_HANDLING = 0, SPECIAL_HANDLING = 1 }; -enum class FunctionSideEffects : uint8_t { NO_SIDE_EFFECTS = 0, HAS_SIDE_EFFECTS = 1 }; +//! The stability of the function, used by the optimizer +//! CONSISTENT -> this function always returns the same result when given the same input, no variance +//! CONSISTENT_WITHIN_QUERY -> this function returns the same result WITHIN the same query/transaction +//! but the result might change across queries (e.g. NOW(), CURRENT_TIME) +//! VOLATILE -> the result of this function might change per row (e.g. RANDOM()) +enum class FunctionStability : uint8_t { CONSISTENT = 0, VOLATILE = 1, CONSISTENT_WITHIN_QUERY = 2 }; struct FunctionData { DUCKDB_API virtual ~FunctionData(); @@ -96,10 +101,11 @@ class Function { public: //! Returns the formatted string name(arg1, arg2, ...) - DUCKDB_API static string CallToString(const string &name, const vector &arguments); + DUCKDB_API static string CallToString(const string &name, const vector &arguments, + const LogicalType &varargs = LogicalType::INVALID); //! Returns the formatted string name(arg1, arg2..) -> return_type DUCKDB_API static string CallToString(const string &name, const vector &arguments, - const LogicalType &return_type); + const LogicalType &varargs, const LogicalType &return_type); //! Returns the formatted string name(arg1, arg2.., np1=a, np2=b, ...) DUCKDB_API static string CallToString(const string &name, const vector &arguments, const named_parameter_type_map_t &named_parameters); @@ -147,16 +153,15 @@ class SimpleNamedParameterFunction : public SimpleFunction { class BaseScalarFunction : public SimpleFunction { public: DUCKDB_API BaseScalarFunction(string name, vector arguments, LogicalType return_type, - FunctionSideEffects side_effects, + FunctionStability stability, LogicalType varargs = LogicalType(LogicalTypeId::INVALID), FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING); DUCKDB_API ~BaseScalarFunction() override; //! Return type of the function LogicalType return_type; - //! Whether or not the function has side effects (e.g. sequence increments, random() functions, NOW()). Functions - //! with side-effects cannot be constant-folded. - FunctionSideEffects side_effects; + //! The stability of the function (see FunctionStability enum for more info) + FunctionStability stability; //! How this function handles NULL values FunctionNullHandling null_handling; diff --git a/src/duckdb/src/include/duckdb/function/function_binder.hpp b/src/duckdb/src/include/duckdb/function/function_binder.hpp index c7e89c94a..2c0929576 100644 --- a/src/duckdb/src/include/duckdb/function/function_binder.hpp +++ b/src/duckdb/src/include/duckdb/function/function_binder.hpp @@ -13,6 +13,8 @@ #include "duckdb/function/scalar_function.hpp" #include "duckdb/function/aggregate_function.hpp" #include "duckdb/function/function_set.hpp" +#include "duckdb/common/exception/binder_exception.hpp" +#include "duckdb/common/error_data.hpp" namespace duckdb { @@ -27,30 +29,30 @@ class FunctionBinder { //! Bind a scalar function from the set of functions and input arguments. Returns the index of the chosen function, //! returns DConstants::INVALID_INDEX and sets error if none could be found DUCKDB_API idx_t BindFunction(const string &name, ScalarFunctionSet &functions, - const vector &arguments, string &error); + const vector &arguments, ErrorData &error); DUCKDB_API idx_t BindFunction(const string &name, ScalarFunctionSet &functions, - vector> &arguments, string &error); + vector> &arguments, ErrorData &error); //! Bind an aggregate function from the set of functions and input arguments. Returns the index of the chosen //! function, returns DConstants::INVALID_INDEX and sets error if none could be found DUCKDB_API idx_t BindFunction(const string &name, AggregateFunctionSet &functions, - const vector &arguments, string &error); + const vector &arguments, ErrorData &error); DUCKDB_API idx_t BindFunction(const string &name, AggregateFunctionSet &functions, - vector> &arguments, string &error); + vector> &arguments, ErrorData &error); //! Bind a table function from the set of functions and input arguments. Returns the index of the chosen //! function, returns DConstants::INVALID_INDEX and sets error if none could be found DUCKDB_API idx_t BindFunction(const string &name, TableFunctionSet &functions, const vector &arguments, - string &error); + ErrorData &error); DUCKDB_API idx_t BindFunction(const string &name, TableFunctionSet &functions, - vector> &arguments, string &error); + vector> &arguments, ErrorData &error); //! Bind a pragma function from the set of functions and input arguments DUCKDB_API idx_t BindFunction(const string &name, PragmaFunctionSet &functions, vector ¶meters, - string &error); + ErrorData &error); DUCKDB_API unique_ptr BindScalarFunction(const string &schema, const string &name, - vector> children, string &error, + vector> children, ErrorData &error, bool is_operator = false, Binder *binder = nullptr); DUCKDB_API unique_ptr BindScalarFunction(ScalarFunctionCatalogEntry &function, - vector> children, string &error, + vector> children, ErrorData &error, bool is_operator = false, Binder *binder = nullptr); DUCKDB_API unique_ptr BindScalarFunction(ScalarFunction bound_function, @@ -73,15 +75,15 @@ class FunctionBinder { template vector BindFunctionsFromArguments(const string &name, FunctionSet &functions, - const vector &arguments, string &error); + const vector &arguments, ErrorData &error); template idx_t MultipleCandidateException(const string &name, FunctionSet &functions, vector &candidate_functions, - const vector &arguments, string &error); + const vector &arguments, ErrorData &error); template idx_t BindFunctionFromArguments(const string &name, FunctionSet &functions, const vector &arguments, - string &error); + ErrorData &error); vector GetLogicalTypesFromExpressions(vector> &arguments); }; diff --git a/src/duckdb/src/include/duckdb/function/function_serialization.hpp b/src/duckdb/src/include/duckdb/function/function_serialization.hpp index ac244f385..fc33f5dc0 100644 --- a/src/duckdb/src/include/duckdb/function/function_serialization.hpp +++ b/src/duckdb/src/include/duckdb/function/function_serialization.hpp @@ -86,9 +86,10 @@ class FunctionSerializer { } else if (function.bind) { try { bind_data = function.bind(context, function, children); - } catch (Exception &ex) { - // FIXME - throw SerializationException("Error during bind of function in deserialization: %s", ex.what()); + } catch (std::exception &ex) { + ErrorData error(ex); + throw SerializationException("Error during bind of function in deserialization: %s", + error.RawMessage()); } } function.return_type = std::move(return_type); diff --git a/src/duckdb/src/include/duckdb/function/macro_function.hpp b/src/duckdb/src/include/duckdb/function/macro_function.hpp index 0a9bdb5a5..e6a0f1854 100644 --- a/src/duckdb/src/include/duckdb/function/macro_function.hpp +++ b/src/duckdb/src/include/duckdb/function/macro_function.hpp @@ -8,12 +8,12 @@ #pragma once -#include "duckdb/parser/query_node.hpp" #include "duckdb/function/function.hpp" #include "duckdb/main/client_context.hpp" +#include "duckdb/parser/expression/constant_expression.hpp" +#include "duckdb/parser/query_node.hpp" #include "duckdb/planner/binder.hpp" #include "duckdb/planner/expression_binder.hpp" -#include "duckdb/parser/expression/constant_expression.hpp" namespace duckdb { @@ -28,7 +28,7 @@ class MacroFunction { //! The positional parameters vector> parameters; //! The default parameters and their associated values - unordered_map> default_parameters; + case_insensitive_map_t> default_parameters; public: virtual ~MacroFunction() { diff --git a/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp b/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp index 5989d6de7..b5125cdc7 100644 --- a/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/pragma/pragma_functions.hpp @@ -21,6 +21,9 @@ struct PragmaFunctions { static void RegisterFunction(BuiltinFunctions &set); }; -string PragmaShow(ClientContext &context, const FunctionParameters ¶meters); +string PragmaShowTables(); +string PragmaShowTablesExpanded(); +string PragmaShowDatabases(); +string PragmaShow(const string &table_name); } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/scalar/list/contains_or_position.hpp b/src/duckdb/src/include/duckdb/function/scalar/list/contains_or_position.hpp index 0eceb782e..c05e98a53 100644 --- a/src/duckdb/src/include/duckdb/function/scalar/list/contains_or_position.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar/list/contains_or_position.hpp @@ -113,6 +113,9 @@ void ListContainsOrPosition(DataChunk &args, Vector &result) { case PhysicalType::UINT64: TemplatedContainsOrPosition(args, result); break; + case PhysicalType::UINT128: + TemplatedContainsOrPosition(args, result); + break; case PhysicalType::FLOAT: TemplatedContainsOrPosition(args, result); break; diff --git a/src/duckdb/src/include/duckdb/function/scalar/sequence_functions.hpp b/src/duckdb/src/include/duckdb/function/scalar/sequence_functions.hpp index ee21a651b..7323ea037 100644 --- a/src/duckdb/src/include/duckdb/function/scalar/sequence_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar/sequence_functions.hpp @@ -8,12 +8,34 @@ #pragma once +#include "duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp" #include "duckdb/function/scalar_function.hpp" #include "duckdb/function/function_set.hpp" #include "duckdb/function/built_in_functions.hpp" namespace duckdb { +struct NextvalBindData : public FunctionData { + explicit NextvalBindData(optional_ptr sequence) + : sequence(sequence), create_info(sequence ? sequence->GetInfo() : nullptr) { + } + + //! The sequence to use for the nextval computation; only if the sequence is a constant + optional_ptr sequence; + + //! The CreateInfo for the above sequence, if it exists + unique_ptr create_info; + + unique_ptr Copy() const override { + return make_uniq(sequence); + } + + bool Equals(const FunctionData &other_p) const override { + auto &other = other_p.Cast(); + return sequence == other.sequence; + } +}; + struct NextvalFun { 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 caa4cc634..b9807b007 100644 --- a/src/duckdb/src/include/duckdb/function/scalar_function.hpp +++ b/src/duckdb/src/include/duckdb/function/scalar_function.hpp @@ -78,7 +78,7 @@ class ScalarFunction : public BaseScalarFunction { 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, + FunctionStability stability = FunctionStability::CONSISTENT, FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING, bind_lambda_function_t bind_lambda = nullptr); @@ -86,7 +86,7 @@ class ScalarFunction : public BaseScalarFunction { 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, + FunctionStability stability = FunctionStability::CONSISTENT, FunctionNullHandling null_handling = FunctionNullHandling::DEFAULT_NULL_HANDLING, bind_lambda_function_t bind_lambda = nullptr); @@ -165,6 +165,9 @@ class ScalarFunction : public BaseScalarFunction { case LogicalTypeId::HUGEINT: function = &ScalarFunction::UnaryFunction; break; + case LogicalTypeId::UHUGEINT: + function = &ScalarFunction::UnaryFunction; + break; case LogicalTypeId::FLOAT: function = &ScalarFunction::UnaryFunction; break; @@ -208,6 +211,9 @@ class ScalarFunction : public BaseScalarFunction { case LogicalTypeId::HUGEINT: function = &ScalarFunction::UnaryFunction; break; + case LogicalTypeId::UHUGEINT: + function = &ScalarFunction::UnaryFunction; + break; case LogicalTypeId::FLOAT: function = &ScalarFunction::UnaryFunction; break; diff --git a/src/duckdb/src/include/duckdb/function/table/arrow.hpp b/src/duckdb/src/include/duckdb/function/table/arrow.hpp index 930212d91..3750a6c7e 100644 --- a/src/duckdb/src/include/duckdb/function/table/arrow.hpp +++ b/src/duckdb/src/include/duckdb/function/table/arrow.hpp @@ -63,6 +63,22 @@ struct ArrowScanFunctionData : public PyTableFunctionData { ArrowTableType arrow_table; }; +struct ArrowRunEndEncodingState { +public: + ArrowRunEndEncodingState() { + } + +public: + unique_ptr run_ends; + unique_ptr values; + +public: + void Reset() { + run_ends.reset(); + values.reset(); + } +}; + struct ArrowScanLocalState; struct ArrowArrayScanState { public: @@ -73,12 +89,24 @@ struct ArrowArrayScanState { unordered_map> children; // Cache the (optional) dictionary of this array unique_ptr dictionary; + //! Run-end-encoding state + ArrowRunEndEncodingState run_end_encoding; public: ArrowArrayScanState &GetChild(idx_t child_idx); void AddDictionary(unique_ptr dictionary_p); bool HasDictionary() const; Vector &GetDictionary(); + ArrowRunEndEncodingState &RunEndEncoding() { + return run_end_encoding; + } + +public: + void Reset() { + // Note: dictionary is not reset + // the dictionary should be the same for every array scanned of this column + run_end_encoding.Reset(); + } }; struct ArrowScanLocalState : public LocalTableFunctionState { @@ -101,6 +129,12 @@ struct ArrowScanLocalState : public LocalTableFunctionState { DataChunk all_columns; public: + void Reset() { + chunk_offset = 0; + for (auto &col : array_states) { + col.second->Reset(); + } + } ArrowArrayScanState &GetState(idx_t child_idx) { auto it = array_states.find(child_idx); if (it == array_states.end()) { diff --git a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp index d475875f0..089785596 100644 --- a/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp +++ b/src/duckdb/src/include/duckdb/function/table/arrow/arrow_duck_schema.hpp @@ -69,8 +69,12 @@ class ArrowType { ArrowDateTimeType GetDateTimeType() const; + void SetRunEndEncoded(); + const ArrowType &GetDictionary() const; + bool RunEndEncoded() const; + const ArrowType &operator[](idx_t index) const; private: @@ -85,6 +89,8 @@ class ArrowType { idx_t fixed_size = 0; //! Hold the optional type if the array is a dictionary unique_ptr dictionary_type; + //! Is run-end-encoded + bool run_end_encoded = false; }; using arrow_column_map_t = unordered_map>; diff --git a/src/duckdb/src/include/duckdb/function/table/list.hpp b/src/duckdb/src/include/duckdb/function/table/list.hpp index efeade1d7..da9633d98 100644 --- a/src/duckdb/src/include/duckdb/function/table/list.hpp +++ b/src/duckdb/src/include/duckdb/function/table/list.hpp @@ -2,3 +2,4 @@ #include "duckdb/function/table/system_functions.hpp" #include "duckdb/function/table/range.hpp" #include "duckdb/function/table/summary.hpp" +#include "duckdb/function/table/files.hpp" diff --git a/src/duckdb/src/include/duckdb/function/table/range.hpp b/src/duckdb/src/include/duckdb/function/table/range.hpp index 298df1044..fa69d615d 100644 --- a/src/duckdb/src/include/duckdb/function/table/range.hpp +++ b/src/duckdb/src/include/duckdb/function/table/range.hpp @@ -41,4 +41,12 @@ struct CSVSnifferFunction { static void RegisterFunction(BuiltinFunctions &set); }; +struct ReadBlobFunction { + static void RegisterFunction(BuiltinFunctions &set); +}; + +struct ReadTextFunction { + static void RegisterFunction(BuiltinFunctions &set); +}; + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/function/table/read_csv.hpp b/src/duckdb/src/include/duckdb/function/table/read_csv.hpp index d0b67f2dc..20d70c63e 100644 --- a/src/duckdb/src/include/duckdb/function/table/read_csv.hpp +++ b/src/duckdb/src/include/duckdb/function/table/read_csv.hpp @@ -8,18 +8,20 @@ #pragma once -#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_file_handle.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" -#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" +#include "duckdb/common/multi_file_reader.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_handle.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp" #include "duckdb/function/built_in_functions.hpp" #include "duckdb/function/scalar/strftime_format.hpp" #include "duckdb/function/table_function.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_file_scanner.hpp" namespace duckdb { +class BaseScanner; +class StringValueScanner; class ReadCSV { public: @@ -73,6 +75,7 @@ struct ColumnInfo { }; struct ReadCSVData : public BaseCSVData { + ReadCSVData(); //! The expected SQL types to read from the file vector csv_types; //! The expected SQL names to be read from the file @@ -84,21 +87,15 @@ struct ReadCSVData : public BaseCSVData { //! The buffer manager (if any): this is used when automatic detection is used during binding. //! In this case, some CSV buffers have already been read and can be reused. shared_ptr buffer_manager; - unique_ptr initial_reader; + unique_ptr initial_reader; //! The union readers are created (when csv union_by_name option is on) during binding //! Those readers can be re-used during ReadCSVFunction - vector> union_readers; - //! Whether or not the single-threaded reader should be used - bool single_threaded = false; + vector> union_readers; //! Reader bind data MultiFileReaderBindData reader_bind; vector column_info; - //! The CSVStateMachineCache caches state machines created for sniffing and parsing csv files - //! We cache them because when reading very small csv files, the cost of creating all the possible - //! State machines for sniffing becomes a major bottleneck. - CSVStateMachineCache state_machine_cache; - void Initialize(unique_ptr &reader) { + void Initialize(unique_ptr &reader) { this->initial_reader = std::move(reader); } void FinalizeRead(ClientContext &context); 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 82e3bc65a..1a2928308 100644 --- a/src/duckdb/src/include/duckdb/function/table/system_functions.hpp +++ b/src/duckdb/src/include/duckdb/function/table/system_functions.hpp @@ -61,6 +61,10 @@ struct DuckDBConstraintsFun { static void RegisterFunction(BuiltinFunctions &set); }; +struct DuckDBSecretsFun { + static void RegisterFunction(BuiltinFunctions &set); +}; + struct DuckDBDatabasesFun { static void RegisterFunction(BuiltinFunctions &set); }; diff --git a/src/duckdb/src/include/duckdb/main/appender.hpp b/src/duckdb/src/include/duckdb/main/appender.hpp index 4fa513a79..b966b6c55 100644 --- a/src/duckdb/src/include/duckdb/main/appender.hpp +++ b/src/duckdb/src/include/duckdb/main/appender.hpp @@ -60,7 +60,7 @@ class BaseAppender { // Append functions template void Append(T value) { - throw Exception("Undefined type for Appender::Append!"); + throw InternalException("Undefined type for Appender::Append!"); } DUCKDB_API void Append(const char *value, uint32_t length); @@ -153,6 +153,8 @@ DUCKDB_API void BaseAppender::Append(int64_t value); template <> DUCKDB_API void BaseAppender::Append(hugeint_t value); template <> +DUCKDB_API void BaseAppender::Append(uhugeint_t value); +template <> DUCKDB_API void BaseAppender::Append(uint8_t value); template <> DUCKDB_API void BaseAppender::Append(uint16_t value); diff --git a/src/duckdb/src/include/duckdb/main/capi/cast/generic.hpp b/src/duckdb/src/include/duckdb/main/capi/cast/generic.hpp index 3aa708f4a..ac01ec750 100644 --- a/src/duckdb/src/include/duckdb/main/capi/cast/generic.hpp +++ b/src/duckdb/src/include/duckdb/main/capi/cast/generic.hpp @@ -54,6 +54,8 @@ RESULT_TYPE GetInternalCValue(duckdb_result *result, idx_t col, idx_t row) { return TryCastCInternal(result, col, row); case DUCKDB_TYPE_HUGEINT: return TryCastCInternal(result, col, row); + case DUCKDB_TYPE_UHUGEINT: + return TryCastCInternal(result, col, row); case DUCKDB_TYPE_DECIMAL: return TryCastDecimalCInternal(result, col, row); case DUCKDB_TYPE_INTERVAL: diff --git a/src/duckdb/src/include/duckdb/main/chunk_scan_state.hpp b/src/duckdb/src/include/duckdb/main/chunk_scan_state.hpp index 8849b09b7..68f113ffc 100644 --- a/src/duckdb/src/include/duckdb/main/chunk_scan_state.hpp +++ b/src/duckdb/src/include/duckdb/main/chunk_scan_state.hpp @@ -2,7 +2,7 @@ #include "duckdb/common/vector.hpp" #include "duckdb/common/unique_ptr.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" namespace duckdb { @@ -21,9 +21,9 @@ class ChunkScanState { ChunkScanState &operator=(ChunkScanState &&other) = default; public: - virtual bool LoadNextChunk(PreservedError &error) = 0; + virtual bool LoadNextChunk(ErrorData &error) = 0; virtual bool HasError() const = 0; - virtual PreservedError &GetError() = 0; + virtual ErrorData &GetError() = 0; virtual const vector &Types() const = 0; virtual const vector &Names() const = 0; idx_t CurrentOffset() const; diff --git a/src/duckdb/src/include/duckdb/main/chunk_scan_state/query_result.hpp b/src/duckdb/src/include/duckdb/main/chunk_scan_state/query_result.hpp index d6f21a507..acdb1086f 100644 --- a/src/duckdb/src/include/duckdb/main/chunk_scan_state/query_result.hpp +++ b/src/duckdb/src/include/duckdb/main/chunk_scan_state/query_result.hpp @@ -1,7 +1,7 @@ #pragma once #include "duckdb/main/chunk_scan_state.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" namespace duckdb { @@ -13,14 +13,14 @@ class QueryResultChunkScanState : public ChunkScanState { ~QueryResultChunkScanState(); public: - bool LoadNextChunk(PreservedError &error) override; + bool LoadNextChunk(ErrorData &error) override; bool HasError() const override; - PreservedError &GetError() override; + ErrorData &GetError() override; const vector &Types() const override; const vector &Names() const override; private: - bool InternalLoad(PreservedError &error); + bool InternalLoad(ErrorData &error); private: QueryResult &result; diff --git a/src/duckdb/src/include/duckdb/main/client_config.hpp b/src/duckdb/src/include/duckdb/main/client_config.hpp index c49dd9a00..92f256cf4 100644 --- a/src/duckdb/src/include/duckdb/main/client_config.hpp +++ b/src/duckdb/src/include/duckdb/main/client_config.hpp @@ -59,6 +59,8 @@ struct ClientConfig { bool query_verification_enabled = false; //! Whether or not verification of external operators is enabled, used for testing bool verify_external = false; + //! Whether or not verification of fetch row code is enabled, used for testing + bool verify_fetch_row = false; //! Whether or not we should verify the serializer bool verify_serializer = false; //! Enable the running of optimizers @@ -73,6 +75,8 @@ struct ClientConfig { bool force_no_cross_product = false; //! Force use of IEJoin to implement AsOfJoin, used for testing bool force_asof_iejoin = false; + //! Force use of fetch row instead of scan, used for testing + bool force_fetch_row = false; //! Use range joins for inequalities, even if there are equality predicates bool prefer_range_joins = false; //! If this context should also try to use the available replacement scans @@ -87,11 +91,6 @@ struct ClientConfig { //! Callback to create a progress bar display progress_bar_display_create_func_t display_create_func = nullptr; - //! Override for the default extension repository - string custom_extension_repo = ""; - //! Override for the default autoload extensoin repository - string autoinstall_extension_repo = ""; - //! The explain output type used when none is specified (default: PHYSICAL_ONLY) ExplainOutputType explain_output_type = ExplainOutputType::PHYSICAL_ONLY; @@ -104,6 +103,9 @@ struct ClientConfig { //! Whether or not the "/" division operator defaults to integer division or floating point division bool integer_division = false; + //! Output error messages as structured JSON instead of as a raw string + bool errors_as_json = false; + //! Generic options case_insensitive_map_t set_variables; @@ -116,7 +118,7 @@ struct ClientConfig { static const ClientConfig &GetConfig(const ClientContext &context); bool AnyVerification() { - return query_verification_enabled || verify_external || verify_serializer; + return query_verification_enabled || verify_external || verify_serializer || verify_fetch_row; } }; diff --git a/src/duckdb/src/include/duckdb/main/client_context.hpp b/src/duckdb/src/include/duckdb/main/client_context.hpp index e33f3eb62..e8c54ab65 100644 --- a/src/duckdb/src/include/duckdb/main/client_context.hpp +++ b/src/duckdb/src/include/duckdb/main/client_context.hpp @@ -23,7 +23,7 @@ #include "duckdb/common/atomic.hpp" #include "duckdb/main/client_config.hpp" #include "duckdb/main/external_dependencies.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/main/client_properties.hpp" namespace duckdb { @@ -193,10 +193,13 @@ class ClientContext : public std::enable_shared_from_this { //! Returns true if execution of the current query is finished DUCKDB_API bool ExecutionIsFinished(); + //! Process an error for display to the user + DUCKDB_API void ProcessError(ErrorData &error, const string &query) const; + private: //! Parse statements and resolve pragmas from a query bool ParseStatements(ClientContextLock &lock, const string &query, vector> &result, - PreservedError &error); + ErrorData &error); //! Issues a query to the database and returns a Pending Query Result unique_ptr PendingQueryInternal(ClientContextLock &lock, unique_ptr statement, const PendingQueryParameters ¶meters, bool verify = true); @@ -206,7 +209,7 @@ class ClientContext : public std::enable_shared_from_this { vector> ParseStatementsInternal(ClientContextLock &lock, const string &query); //! Perform aggressive query verification of a SELECT statement. Only called when query_verification_enabled is //! true. - PreservedError VerifyQuery(ClientContextLock &lock, const string &query, unique_ptr statement); + ErrorData VerifyQuery(ClientContextLock &lock, const string &query, unique_ptr statement); void InitialCleanup(ClientContextLock &lock); //! Internal clean up, does not lock. Caller must hold the context_lock. @@ -240,7 +243,7 @@ class ClientContext : public std::enable_shared_from_this { void BeginTransactionInternal(ClientContextLock &lock, bool requires_valid_transaction); void BeginQueryInternal(ClientContextLock &lock, const string &query); - PreservedError EndQueryInternal(ClientContextLock &lock, bool success, bool invalidate_transaction); + ErrorData EndQueryInternal(ClientContextLock &lock, bool success, bool invalidate_transaction); PendingExecutionResult ExecuteTaskInternal(ClientContextLock &lock, PendingQueryResult &result); @@ -255,6 +258,9 @@ class ClientContext : public std::enable_shared_from_this { unique_ptr PendingQueryInternal(ClientContextLock &, const shared_ptr &relation, bool allow_stream_result); + template + unique_ptr ErrorResult(ErrorData error, const string &query = string()); + private: //! Lock on using the ClientContext in parallel mutex context_lock; diff --git a/src/duckdb/src/include/duckdb/main/client_data.hpp b/src/duckdb/src/include/duckdb/main/client_data.hpp index 4eb385784..bc94d757a 100644 --- a/src/duckdb/src/include/duckdb/main/client_data.hpp +++ b/src/duckdb/src/include/duckdb/main/client_data.hpp @@ -13,7 +13,7 @@ #include "duckdb/common/types/value.hpp" #include "duckdb/common/case_insensitive_map.hpp" #include "duckdb/common/atomic.hpp" -#include "duckdb/execution/operator/scan/csv/csv_state_machine_cache.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_state_machine_cache.hpp" namespace duckdb { class AttachedDatabase; @@ -54,9 +54,6 @@ struct ClientData { //! The file opener of the client context unique_ptr file_opener; - //! HTTP State in this query - shared_ptr http_state; - //! The clients' file system wrapper unique_ptr client_file_system; diff --git a/src/duckdb/src/include/duckdb/main/config.hpp b/src/duckdb/src/include/duckdb/main/config.hpp index bb52d7747..8f0bc7470 100644 --- a/src/duckdb/src/include/duckdb/main/config.hpp +++ b/src/duckdb/src/include/duckdb/main/config.hpp @@ -24,13 +24,16 @@ #include "duckdb/common/winapi.hpp" #include "duckdb/function/cast/default_casts.hpp" #include "duckdb/function/replacement_scan.hpp" -#include "duckdb/main/client_properties.hpp" #include "duckdb/optimizer/optimizer_extension.hpp" +#include "duckdb/parser/parsed_data/create_info.hpp" #include "duckdb/parser/parser_extension.hpp" #include "duckdb/planner/operator_extension.hpp" #include "duckdb/storage/compression/bitpacking.hpp" +#include "duckdb/main/client_properties.hpp" +#include "duckdb/execution/index/index_type_set.hpp" namespace duckdb { + class BufferPool; class CastFunctionSet; class ClientContext; @@ -40,6 +43,7 @@ class TableFunctionRef; class OperatorExtension; class StorageExtension; class ExtensionCallback; +class SecretManager; struct CompressionFunctionSet; struct DBConfig; @@ -108,6 +112,10 @@ struct DBConfigOptions { #else bool autoinstall_known_extensions = false; #endif + //! Override for the default extension repository + string custom_extension_repo = ""; + //! Override for the default autoload extensoin repository + string autoinstall_extension_repo = ""; //! The maximum memory used by the database system (in bytes). Default: 80% of System available memory idx_t maximum_memory = (idx_t)-1; //! The maximum amount of CPU threads used by the database system. Default: all available. @@ -118,6 +126,8 @@ struct DBConfigOptions { bool use_temporary_directory = true; //! Directory to store temporary structures that do not fit in memory string temporary_directory; + //! Whether or not to allow printing unredacted secrets + bool allow_unredacted_secrets = false; //! The collation type of the database string collation = string(); //! The order type used when none is specified (default: ASC) @@ -177,6 +187,8 @@ struct DBConfigOptions { string duckdb_api; //! Metadata from DuckDB callers string custom_user_agent; + //! Use old implicit casting style (i.e. allow everything to be implicitly casted to VARCHAR) + bool old_implicit_casting = false; bool operator==(const DBConfigOptions &other) const; }; @@ -200,6 +212,8 @@ struct DBConfig { //! The FileSystem to use, can be overwritten to allow for injecting custom file systems for testing purposes (e.g. //! RamFS or something similar) unique_ptr file_system; + //! Secret manager + unique_ptr secret_manager; //! The allocator used by the system unique_ptr allocator; //! Database configuration options @@ -237,7 +251,6 @@ struct DBConfig { DUCKDB_API static ConfigurationOption *GetOptionByIndex(idx_t index); //! Fetch an option by name. Returns a pointer to the option, or nullptr if none exists. DUCKDB_API static ConfigurationOption *GetOptionByName(const string &name); - DUCKDB_API void SetOption(const ConfigurationOption &option, const Value &value); DUCKDB_API void SetOption(DatabaseInstance *db, const ConfigurationOption &option, const Value &value); DUCKDB_API void SetOptionByName(const string &name, const Value &value); @@ -259,6 +272,7 @@ struct DBConfig { bool operator!=(const DBConfig &other); DUCKDB_API CastFunctionSet &GetCastFunctions(); + DUCKDB_API IndexTypeSet &GetIndexTypes(); static idx_t GetSystemMaxThreads(FileSystem &fs); void SetDefaultMaxThreads(); void SetDefaultMaxMemory(); @@ -270,6 +284,7 @@ struct DBConfig { private: unique_ptr compression_functions; unique_ptr cast_functions; + unique_ptr index_types; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/database.hpp b/src/duckdb/src/include/duckdb/main/database.hpp index f6c6302ff..35aaf6d85 100644 --- a/src/duckdb/src/include/duckdb/main/database.hpp +++ b/src/duckdb/src/include/duckdb/main/database.hpp @@ -35,7 +35,8 @@ class DatabaseInstance : public std::enable_shared_from_this { DBConfig config; public: - BufferPool &GetBufferPool(); + BufferPool &GetBufferPool() const; + DUCKDB_API SecretManager &GetSecretManager(); DUCKDB_API BufferManager &GetBufferManager(); DUCKDB_API DatabaseManager &GetDatabaseManager(); DUCKDB_API FileSystem &GetFileSystem(); diff --git a/src/duckdb/src/include/duckdb/main/error_manager.hpp b/src/duckdb/src/include/duckdb/main/error_manager.hpp index 2fd63b365..a0d1c229f 100644 --- a/src/duckdb/src/include/duckdb/main/error_manager.hpp +++ b/src/duckdb/src/include/duckdb/main/error_manager.hpp @@ -15,6 +15,7 @@ namespace duckdb { class ClientContext; class DatabaseInstance; +class TransactionException; enum class ErrorType : uint16_t { // error message types @@ -51,7 +52,9 @@ class ErrorManager { return Get(context).FormatException(error_type, params...); } - DUCKDB_API static string InvalidUnicodeError(const string &input, const string &context); + DUCKDB_API static InvalidInputException InvalidUnicodeError(const string &input, const string &context); + DUCKDB_API static FatalException InvalidatedDatabase(ClientContext &context, const string &invalidated_msg); + DUCKDB_API static TransactionException InvalidatedTransaction(ClientContext &context); //! Adds a custom error for a specific error type void AddCustomError(ErrorType type, string new_error); diff --git a/src/duckdb/src/include/duckdb/main/extension_entries.hpp b/src/duckdb/src/include/duckdb/main/extension_entries.hpp index a3d5936d0..1c96337d7 100644 --- a/src/duckdb/src/include/duckdb/main/extension_entries.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_entries.hpp @@ -20,190 +20,190 @@ struct ExtensionEntry { char extension[48]; }; -static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = { - {"->>", "json"}, - {"add_parquet_key", "parquet"}, - {"array_to_json", "json"}, - {"create_fts_index", "fts"}, - {"current_localtime", "icu"}, - {"current_localtimestamp", "icu"}, - {"dbgen", "tpch"}, - {"drop_fts_index", "fts"}, - {"dsdgen", "tpcds"}, - {"excel_text", "excel"}, - {"from_json", "json"}, - {"from_json_strict", "json"}, - {"from_substrait", "substrait"}, - {"from_substrait_json", "substrait"}, - {"fuzz_all_functions", "sqlsmith"}, - {"fuzzyduck", "sqlsmith"}, - {"get_substrait", "substrait"}, - {"get_substrait_json", "substrait"}, - {"host", "inet"}, - {"iceberg_metadata", "iceberg"}, - {"iceberg_scan", "iceberg"}, - {"iceberg_snapshots", "iceberg"}, - {"icu_calendar_names", "icu"}, - {"icu_sort_key", "icu"}, - {"json", "json"}, - {"json_array", "json"}, - {"json_array_length", "json"}, - {"json_contains", "json"}, - {"json_deserialize_sql", "json"}, - {"json_execute_serialized_sql", "json"}, - {"json_extract", "json"}, - {"json_extract_path", "json"}, - {"json_extract_path_text", "json"}, - {"json_extract_string", "json"}, - {"json_group_array", "json"}, - {"json_group_object", "json"}, - {"json_group_structure", "json"}, - {"json_keys", "json"}, - {"json_merge_patch", "json"}, - {"json_object", "json"}, - {"json_quote", "json"}, - {"json_serialize_plan", "json"}, - {"json_serialize_sql", "json"}, - {"json_structure", "json"}, - {"json_transform", "json"}, - {"json_transform_strict", "json"}, - {"json_type", "json"}, - {"json_valid", "json"}, - {"load_aws_credentials", "aws"}, - {"make_timestamptz", "icu"}, - {"parquet_file_metadata", "parquet"}, - {"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"}, - {"read_json_auto", "json"}, - {"read_json_objects", "json"}, - {"read_json_objects_auto", "json"}, - {"read_ndjson", "json"}, - {"read_ndjson_auto", "json"}, - {"read_ndjson_objects", "json"}, - {"read_parquet", "parquet"}, - {"reduce_sql_statement", "sqlsmith"}, - {"row_to_json", "json"}, - {"scan_arrow_ipc", "arrow"}, - {"sql_auto_complete", "autocomplete"}, - {"sqlite_attach", "sqlite_scanner"}, - {"sqlite_scan", "sqlite_scanner"}, - {"sqlsmith", "sqlsmith"}, - {"st_area", "spatial"}, - {"st_area_spheroid", "spatial"}, - {"st_asgeojson", "spatial"}, - {"st_ashexwkb", "spatial"}, - {"st_astext", "spatial"}, - {"st_aswkb", "spatial"}, - {"st_boundary", "spatial"}, - {"st_buffer", "spatial"}, - {"st_centroid", "spatial"}, - {"st_collect", "spatial"}, - {"st_collectionextract", "spatial"}, - {"st_contains", "spatial"}, - {"st_containsproperly", "spatial"}, - {"st_convexhull", "spatial"}, - {"st_coveredby", "spatial"}, - {"st_covers", "spatial"}, - {"st_crosses", "spatial"}, - {"st_difference", "spatial"}, - {"st_dimension", "spatial"}, - {"st_disjoint", "spatial"}, - {"st_distance", "spatial"}, - {"st_distance_spheroid", "spatial"}, - {"st_drivers", "spatial"}, - {"st_dump", "spatial"}, - {"st_dwithin", "spatial"}, - {"st_dwithin_spheroid", "spatial"}, - {"st_endpoint", "spatial"}, - {"st_envelope", "spatial"}, - {"st_envelope_agg", "spatial"}, - {"st_equals", "spatial"}, - {"st_extent", "spatial"}, - {"st_exteriorring", "spatial"}, - {"st_flipcoordinates", "spatial"}, - {"st_geometrytype", "spatial"}, - {"st_geomfromgeojson", "spatial"}, - {"st_geomfromhexewkb", "spatial"}, - {"st_geomfromhexwkb", "spatial"}, - {"st_geomfromtext", "spatial"}, - {"st_geomfromwkb", "spatial"}, - {"st_intersection", "spatial"}, - {"st_intersection_agg", "spatial"}, - {"st_intersects", "spatial"}, - {"st_intersects_extent", "spatial"}, - {"st_isclosed", "spatial"}, - {"st_isempty", "spatial"}, - {"st_isring", "spatial"}, - {"st_issimple", "spatial"}, - {"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"}, - {"st_npoints", "spatial"}, - {"st_numgeometries", "spatial"}, - {"st_numinteriorrings", "spatial"}, - {"st_numpoints", "spatial"}, - {"st_overlaps", "spatial"}, - {"st_perimeter", "spatial"}, - {"st_perimeter_spheroid", "spatial"}, - {"st_point", "spatial"}, - {"st_point2d", "spatial"}, - {"st_point2dfromwkb", "spatial"}, - {"st_point3d", "spatial"}, - {"st_point4d", "spatial"}, - {"st_pointn", "spatial"}, - {"st_pointonsurface", "spatial"}, - {"st_polygon2dfromwkb", "spatial"}, - {"st_reverse", "spatial"}, - {"st_read", "spatial"}, - {"st_readosm", "spatial"}, - {"st_reduceprecision", "spatial"}, - {"st_removerepeatedpoints", "spatial"}, - {"st_simplify", "spatial"}, - {"st_simplifypreservetopology", "spatial"}, - {"st_startpoint", "spatial"}, - {"st_touches", "spatial"}, - {"st_transform", "spatial"}, - {"st_union", "spatial"}, - {"st_union_agg", "spatial"}, - {"st_within", "spatial"}, - {"st_x", "spatial"}, - {"st_xmax", "spatial"}, - {"st_xmin", "spatial"}, - {"st_y", "spatial"}, - {"st_ymax", "spatial"}, - {"st_ymin", "spatial"}, - {"stem", "fts"}, - {"text", "excel"}, - {"to_arrow_ipc", "arrow"}, - {"to_json", "json"}, - {"tpcds", "tpcds"}, - {"tpcds_answers", "tpcds"}, - {"tpcds_queries", "tpcds"}, - {"tpch", "tpch"}, - {"tpch_answers", "tpch"}, - {"tpch_queries", "tpch"}, - {"visualize_diff_profiling_output", "visualizer"}, - {"visualize_json_profiling_output", "visualizer"}, - {"visualize_last_profiling_output", "visualizer"}, -}; // END_OF_EXTENSION_FUNCTIONS +static constexpr ExtensionEntry EXTENSION_FUNCTIONS[] = {{"->>", "json"}, + {"add_parquet_key", "parquet"}, + {"array_to_json", "json"}, + {"create_fts_index", "fts"}, + {"current_localtime", "icu"}, + {"current_localtimestamp", "icu"}, + {"dbgen", "tpch"}, + {"drop_fts_index", "fts"}, + {"dsdgen", "tpcds"}, + {"excel_text", "excel"}, + {"from_json", "json"}, + {"from_json_strict", "json"}, + {"from_substrait", "substrait"}, + {"from_substrait_json", "substrait"}, + {"fuzz_all_functions", "sqlsmith"}, + {"fuzzyduck", "sqlsmith"}, + {"get_substrait", "substrait"}, + {"get_substrait_json", "substrait"}, + {"host", "inet"}, + {"iceberg_metadata", "iceberg"}, + {"iceberg_scan", "iceberg"}, + {"iceberg_snapshots", "iceberg"}, + {"icu_calendar_names", "icu"}, + {"icu_sort_key", "icu"}, + {"json", "json"}, + {"json_array", "json"}, + {"json_array_length", "json"}, + {"json_contains", "json"}, + {"json_deserialize_sql", "json"}, + {"json_execute_serialized_sql", "json"}, + {"json_extract", "json"}, + {"json_extract_path", "json"}, + {"json_extract_path_text", "json"}, + {"json_extract_string", "json"}, + {"json_group_array", "json"}, + {"json_group_object", "json"}, + {"json_group_structure", "json"}, + {"json_keys", "json"}, + {"json_merge_patch", "json"}, + {"json_object", "json"}, + {"json_quote", "json"}, + {"json_serialize_plan", "json"}, + {"json_serialize_sql", "json"}, + {"json_structure", "json"}, + {"json_transform", "json"}, + {"json_transform_strict", "json"}, + {"json_type", "json"}, + {"json_valid", "json"}, + {"load_aws_credentials", "aws"}, + {"make_timestamptz", "icu"}, + {"parquet_file_metadata", "parquet"}, + {"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"}, + {"read_json_auto", "json"}, + {"read_json_objects", "json"}, + {"read_json_objects_auto", "json"}, + {"read_ndjson", "json"}, + {"read_ndjson_auto", "json"}, + {"read_ndjson_objects", "json"}, + {"read_parquet", "parquet"}, + {"reduce_sql_statement", "sqlsmith"}, + {"row_to_json", "json"}, + {"scan_arrow_ipc", "arrow"}, + {"sql_auto_complete", "autocomplete"}, + {"sqlite_attach", "sqlite_scanner"}, + {"sqlite_scan", "sqlite_scanner"}, + {"sqlsmith", "sqlsmith"}, + {"shapefile_meta", "spatial"}, + {"st_area", "spatial"}, + {"st_area_spheroid", "spatial"}, + {"st_asgeojson", "spatial"}, + {"st_ashexwkb", "spatial"}, + {"st_astext", "spatial"}, + {"st_aswkb", "spatial"}, + {"st_boundary", "spatial"}, + {"st_buffer", "spatial"}, + {"st_centroid", "spatial"}, + {"st_collect", "spatial"}, + {"st_collectionextract", "spatial"}, + {"st_contains", "spatial"}, + {"st_containsproperly", "spatial"}, + {"st_convexhull", "spatial"}, + {"st_coveredby", "spatial"}, + {"st_covers", "spatial"}, + {"st_crosses", "spatial"}, + {"st_difference", "spatial"}, + {"st_dimension", "spatial"}, + {"st_disjoint", "spatial"}, + {"st_distance", "spatial"}, + {"st_distance_spheroid", "spatial"}, + {"st_drivers", "spatial"}, + {"st_dump", "spatial"}, + {"st_dwithin", "spatial"}, + {"st_dwithin_spheroid", "spatial"}, + {"st_endpoint", "spatial"}, + {"st_envelope", "spatial"}, + {"st_envelope_agg", "spatial"}, + {"st_equals", "spatial"}, + {"st_extent", "spatial"}, + {"st_exteriorring", "spatial"}, + {"st_flipcoordinates", "spatial"}, + {"st_geometrytype", "spatial"}, + {"st_geomfromgeojson", "spatial"}, + {"st_geomfromhexewkb", "spatial"}, + {"st_geomfromhexwkb", "spatial"}, + {"st_geomfromtext", "spatial"}, + {"st_geomfromwkb", "spatial"}, + {"st_intersection", "spatial"}, + {"st_intersection_agg", "spatial"}, + {"st_intersects", "spatial"}, + {"st_intersects_extent", "spatial"}, + {"st_isclosed", "spatial"}, + {"st_isempty", "spatial"}, + {"st_isring", "spatial"}, + {"st_issimple", "spatial"}, + {"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_makevalid", "spatial"}, + {"st_ngeometries", "spatial"}, + {"st_ninteriorrings", "spatial"}, + {"st_normalize", "spatial"}, + {"st_npoints", "spatial"}, + {"st_numgeometries", "spatial"}, + {"st_numinteriorrings", "spatial"}, + {"st_numpoints", "spatial"}, + {"st_overlaps", "spatial"}, + {"st_perimeter", "spatial"}, + {"st_perimeter_spheroid", "spatial"}, + {"st_point", "spatial"}, + {"st_point2d", "spatial"}, + {"st_point2dfromwkb", "spatial"}, + {"st_point3d", "spatial"}, + {"st_point4d", "spatial"}, + {"st_pointn", "spatial"}, + {"st_pointonsurface", "spatial"}, + {"st_polygon2dfromwkb", "spatial"}, + {"st_quadkey", "spatial"}, + {"st_reverse", "spatial"}, + {"st_read", "spatial"}, + {"st_read_meta", "spatial"}, + {"st_readosm", "spatial"}, + {"st_readshp", "spatial"}, + {"st_reduceprecision", "spatial"}, + {"st_removerepeatedpoints", "spatial"}, + {"st_simplify", "spatial"}, + {"st_simplifypreservetopology", "spatial"}, + {"st_startpoint", "spatial"}, + {"st_touches", "spatial"}, + {"st_transform", "spatial"}, + {"st_union", "spatial"}, + {"st_union_agg", "spatial"}, + {"st_within", "spatial"}, + {"st_x", "spatial"}, + {"st_xmax", "spatial"}, + {"st_xmin", "spatial"}, + {"st_y", "spatial"}, + {"st_ymax", "spatial"}, + {"st_ymin", "spatial"}, + {"stem", "fts"}, + {"text", "excel"}, + {"to_arrow_ipc", "arrow"}, + {"to_json", "json"}, + {"tpcds", "tpcds"}, + {"tpcds_answers", "tpcds"}, + {"tpcds_queries", "tpcds"}, + {"tpch", "tpch"}, + {"tpch_answers", "tpch"}, + {"tpch_queries", "tpch"}}; // END_OF_EXTENSION_FUNCTIONS static constexpr ExtensionEntry EXTENSION_SETTINGS[] = { {"azure_storage_connection_string", "azure"}, @@ -218,6 +218,7 @@ static constexpr ExtensionEntry EXTENSION_SETTINGS[] = { {"pg_debug_show_queries", "postgres_scanner"}, {"pg_use_binary_copy", "postgres_scanner"}, {"pg_experimental_filter_pushdown", "postgres_scanner"}, + {"pg_connection_cache", "postgres_scanner"}, {"pg_connection_limit", "postgres_scanner"}, {"pg_pages_per_task", "postgres_scanner"}, {"pg_array_as_varchar", "postgres_scanner"}, @@ -276,8 +277,8 @@ static constexpr ExtensionEntry EXTENSION_COLLATIONS[] = { // TODO: automate by passing though to script via duckdb static constexpr ExtensionEntry EXTENSION_FILE_PREFIXES[] = { {"http://", "httpfs"}, {"https://", "httpfs"}, {"s3://", "httpfs"}, {"s3a://", "httpfs"}, - {"s3n://", "httpfs"}, {"gcs://", "httpfs"}, {"r2://", "httpfs"} // , {"azure://", "azure"} -}; // END_OF_EXTENSION_FILE_PREFIXES + {"s3n://", "httpfs"}, {"gcs://", "httpfs"}, {"gs://", "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 @@ -292,25 +293,25 @@ static constexpr ExtensionEntry EXTENSION_FILE_CONTAINS[] = {{".parquet?", "parq {".ndjson?", ".jsonl?"}, {".jsonl?", ".ndjson?"}}; // EXTENSION_FILE_CONTAINS +// Note: these are currently hardcoded in scripts/generate_extensions_function.py +// TODO: automate by passing though to script via duckdb +static constexpr ExtensionEntry EXTENSION_SECRET_TYPES[] = { + {"s3", "httpfs"}, {"r2", "httpfs"}, {"gcs", "httpfs"}, {"azure", "azure"}}; // EXTENSION_SECRET_TYPES + +// Note: these are currently hardcoded in scripts/generate_extensions_function.py +// TODO: automate by passing though to script via duckdb +static constexpr ExtensionEntry EXTENSION_SECRET_PROVIDERS[] = { + {"s3/config", "httpfs"}, {"gcs/config", "httpfs"}, {"r2/config", "httpfs"}, + {"s3/credential_chain", "aws"}, {"gcs/credential_chain", "aws"}, {"r2/credential_chain", "aws"}, + {"azure/config", "azure"}, {"azure/credential_chain", "azure"}}; // EXTENSION_SECRET_PROVIDERS + static constexpr const char *AUTOLOADABLE_EXTENSIONS[] = { // "azure", - "arrow", - "aws", - "autocomplete", - "excel", - "fts", - "httpfs", + "arrow", "aws", "autocomplete", "excel", "fts", "httpfs", // "inet", // "icu", - "json", - "parquet", - "postgres_scanner", + "json", "parquet", "postgres_scanner", // "spatial", TODO: table function isnt always autoloaded so test fails - "sqlsmith", - "sqlite_scanner", - "tpcds", - "tpch", - "visualizer", -}; // END_OF_AUTOLOADABLE_EXTENSIONS + "sqlsmith", "sqlite_scanner", "tpcds", "tpch"}; // END_OF_AUTOLOADABLE_EXTENSIONS } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/extension_helper.hpp b/src/duckdb/src/include/duckdb/main/extension_helper.hpp index 1d2c5463b..2fc2d5029 100644 --- a/src/duckdb/src/include/duckdb/main/extension_helper.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_helper.hpp @@ -30,7 +30,7 @@ struct ExtensionAlias { struct ExtensionInitResult { string filename; - string basename; + string filebase; void *lib_hdl; }; @@ -41,42 +41,54 @@ class ExtensionHelper { static ExtensionLoadResult LoadExtension(DuckDB &db, const std::string &extension); + //! Install an extension static void InstallExtension(ClientContext &context, const string &extension, bool force_install, const string &respository = ""); static void InstallExtension(DBConfig &config, FileSystem &fs, const string &extension, bool force_install, const string &respository = ""); + //! Load an extension static void LoadExternalExtension(ClientContext &context, const string &extension); - static void LoadExternalExtension(DatabaseInstance &db, FileSystem &fs, const string &extension, - optional_ptr client_config); + static void LoadExternalExtension(DatabaseInstance &db, FileSystem &fs, const string &extension); - //! Autoload an extension by name. Depending on the current settings, this will either load or install+load + //! Autoload an extension (depending on config, potentially a nop. Throws when installation fails) static void AutoLoadExtension(ClientContext &context, const string &extension_name); + static void AutoLoadExtension(DatabaseInstance &db, const string &extension_name); + + //! Autoload an extension (depending on config, potentially a nop. Returns false on failure) DUCKDB_API static bool TryAutoLoadExtension(ClientContext &context, const string &extension_name) noexcept; + //! Get the extension directory base on the current config static string ExtensionDirectory(ClientContext &context); static string ExtensionDirectory(DBConfig &config, FileSystem &fs); - static string ExtensionUrlTemplate(optional_ptr config, const string &repository); + + //! Get the extension url template, containing placeholders for version, platform and extension name + static string ExtensionUrlTemplate(optional_ptr config, const string &repository); + //! Return the extension url template with the variables replaced static string ExtensionFinalizeUrlTemplate(const string &url, const string &name); + //! Default extensions are all extensions that DuckDB knows and expect to be available (both in-tree and + //! out-of-tree) static idx_t DefaultExtensionCount(); static DefaultExtension GetDefaultExtension(idx_t index); + //! Extension can have aliases static idx_t ExtensionAliasCount(); static ExtensionAlias GetExtensionAlias(idx_t index); + //! Get public signing keys for extension signing static const vector GetPublicKeys(); // Returns extension name, or empty string if not a replacement open path static string ExtractExtensionPrefixFromPath(const string &path); - //! Apply any known extension aliases - static string ApplyExtensionAlias(string extension_name); + //! Apply any known extension aliases, return the lowercase name + static string ApplyExtensionAlias(const string &extension_name); static string GetExtensionName(const string &extension); static bool IsFullPath(const string &extension); //! Lookup a name in an ExtensionEntry list - template + template static string FindExtensionInEntries(const string &name, const ExtensionEntry (&entries)[N]) { auto lcase = StringUtil::Lower(name); @@ -89,6 +101,20 @@ class ExtensionHelper { return ""; } + //! Lookup a name in an extension entry and try to autoload it + template + static void TryAutoloadFromEntry(DatabaseInstance &db, const string &entry, const ExtensionEntry (&entries)[N]) { + auto &dbconfig = DBConfig::GetConfig(db); +#ifndef DUCKDB_DISABLE_EXTENSION_LOAD + if (dbconfig.options.autoload_known_extensions) { + auto extension_name = ExtensionHelper::FindExtensionInEntries(entry, entries); + if (ExtensionHelper::CanAutoloadExtension(extension_name)) { + ExtensionHelper::AutoLoadExtension(db, extension_name); + } + } +#endif + } + //! Whether an extension can be autoloaded (i.e. it's registered as an autoloadable extension in //! extension_entries.hpp) static bool CanAutoloadExtension(const string &ext_name); @@ -99,18 +125,18 @@ class ExtensionHelper { static string AddExtensionInstallHintToErrorMsg(ClientContext &context, const string &base_error, const string &extension_name); + //! For tagged releases we use the tag, else we use the git commit hash + static const string GetVersionDirectoryName(); + private: - static void InstallExtensionInternal(DBConfig &config, ClientConfig *client_config, FileSystem &fs, - const string &local_path, const string &extension, bool force_install, - const string &repository); + static void InstallExtensionInternal(DBConfig &config, FileSystem &fs, const string &local_path, + const string &extension, bool force_install, const string &repository); static const vector PathComponents(); + static string DefaultExtensionFolder(FileSystem &fs); static bool AllowAutoInstall(const string &extension); - static ExtensionInitResult InitialLoad(DBConfig &config, FileSystem &fs, const string &extension, - optional_ptr client_config); + static ExtensionInitResult InitialLoad(DBConfig &config, FileSystem &fs, const string &extension); static bool TryInitialLoad(DBConfig &config, FileSystem &fs, const string &extension, ExtensionInitResult &result, - string &error, optional_ptr client_config); - //! For tagged releases we use the tag, else we use the git commit hash - static const string GetVersionDirectoryName(); + string &error); //! Version tags occur with and without 'v', tag in extension path is always with 'v' static const string NormalizeVersionTag(const string &version_tag); static bool IsRelease(const string &version_tag); diff --git a/src/duckdb/src/include/duckdb/main/extension_util.hpp b/src/duckdb/src/include/duckdb/main/extension_util.hpp index 2ada84100..bbfae5f3e 100644 --- a/src/duckdb/src/include/duckdb/main/extension_util.hpp +++ b/src/duckdb/src/include/duckdb/main/extension_util.hpp @@ -11,6 +11,7 @@ #include "duckdb/common/constants.hpp" #include "duckdb/function/cast/cast_function_set.hpp" #include "duckdb/function/function_set.hpp" +#include "duckdb/main/secret/secret.hpp" namespace duckdb { struct CreateMacroInfo; @@ -36,6 +37,10 @@ class ExtensionUtil { DUCKDB_API static void RegisterFunction(DatabaseInstance &db, PragmaFunction function); //! Register a new pragma function set - throw an exception if the function already exists DUCKDB_API static void RegisterFunction(DatabaseInstance &db, PragmaFunctionSet function); + + //! Register a CreateSecretFunction + DUCKDB_API static void RegisterFunction(DatabaseInstance &db, CreateSecretFunction function); + //! Register a new copy function - throw an exception if the function already exists DUCKDB_API static void RegisterFunction(DatabaseInstance &db, CopyFunction function); //! Register a new macro function - throw an exception if the function already exists @@ -51,12 +56,14 @@ class ExtensionUtil { //! Add a function overload DUCKDB_API static void AddFunctionOverload(DatabaseInstance &db, ScalarFunction function); DUCKDB_API static void AddFunctionOverload(DatabaseInstance &db, ScalarFunctionSet function); - DUCKDB_API static void AddFunctionOverload(DatabaseInstance &db, TableFunctionSet function); //! Registers a new type DUCKDB_API static void RegisterType(DatabaseInstance &db, string type_name, LogicalType type); + //! Registers a new secret type + DUCKDB_API static void RegisterSecretType(DatabaseInstance &db, SecretType secret_type); + //! Registers a cast between two types DUCKDB_API static void RegisterCastFunction(DatabaseInstance &db, const LogicalType &source, const LogicalType &target, BoundCastInfo function, diff --git a/src/duckdb/src/include/duckdb/main/materialized_query_result.hpp b/src/duckdb/src/include/duckdb/main/materialized_query_result.hpp index 334a7e991..483f6de1b 100644 --- a/src/duckdb/src/include/duckdb/main/materialized_query_result.hpp +++ b/src/duckdb/src/include/duckdb/main/materialized_query_result.hpp @@ -27,7 +27,7 @@ class MaterializedQueryResult : public QueryResult { vector names, unique_ptr collection, ClientProperties client_properties); //! Creates an unsuccessful query result with error condition - DUCKDB_API explicit MaterializedQueryResult(PreservedError error); + DUCKDB_API explicit MaterializedQueryResult(ErrorData error); public: //! Fetches a DataChunk from the query result. diff --git a/src/duckdb/src/include/duckdb/main/pending_query_result.hpp b/src/duckdb/src/include/duckdb/main/pending_query_result.hpp index 672388c4d..930363e29 100644 --- a/src/duckdb/src/include/duckdb/main/pending_query_result.hpp +++ b/src/duckdb/src/include/duckdb/main/pending_query_result.hpp @@ -26,8 +26,8 @@ class PendingQueryResult : public BaseQueryResult { public: DUCKDB_API PendingQueryResult(shared_ptr context, PreparedStatementData &statement, vector types, bool allow_stream_result); - DUCKDB_API explicit PendingQueryResult(PreservedError error_message); - DUCKDB_API ~PendingQueryResult(); + DUCKDB_API explicit PendingQueryResult(ErrorData error_message); + DUCKDB_API ~PendingQueryResult() override; public: //! Executes a single task within the query, returning whether or not the query is ready. diff --git a/src/duckdb/src/include/duckdb/main/prepared_statement.hpp b/src/duckdb/src/include/duckdb/main/prepared_statement.hpp index 8c5d29c74..d448fdf0c 100644 --- a/src/duckdb/src/include/duckdb/main/prepared_statement.hpp +++ b/src/duckdb/src/include/duckdb/main/prepared_statement.hpp @@ -11,7 +11,7 @@ #include "duckdb/common/winapi.hpp" #include "duckdb/main/materialized_query_result.hpp" #include "duckdb/main/pending_query_result.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/case_insensitive_map.hpp" namespace duckdb { @@ -25,7 +25,7 @@ class PreparedStatement { DUCKDB_API PreparedStatement(shared_ptr context, shared_ptr data, string query, idx_t n_param, case_insensitive_map_t named_param_map); //! Create a prepared statement that was not successfully prepared - DUCKDB_API explicit PreparedStatement(PreservedError error); + DUCKDB_API explicit PreparedStatement(ErrorData error); DUCKDB_API ~PreparedStatement(); @@ -39,7 +39,7 @@ class PreparedStatement { //! Whether or not the statement was successfully prepared bool success; //! The error message (if success = false) - PreservedError error; + ErrorData error; //! The amount of bound parameters idx_t n_param; //! The (optional) named parameters @@ -49,7 +49,7 @@ class PreparedStatement { //! Returns the stored error message DUCKDB_API const string &GetError(); //! Returns the stored error object - DUCKDB_API PreservedError &GetErrorObject(); + DUCKDB_API ErrorData &GetErrorObject(); //! Returns whether or not an error occurred DUCKDB_API bool HasError() const; //! Returns the number of columns in the result diff --git a/src/duckdb/src/include/duckdb/main/query_result.hpp b/src/duckdb/src/include/duckdb/main/query_result.hpp index 3c5088b27..95516053b 100644 --- a/src/duckdb/src/include/duckdb/main/query_result.hpp +++ b/src/duckdb/src/include/duckdb/main/query_result.hpp @@ -11,7 +11,7 @@ #include "duckdb/common/enums/statement_type.hpp" #include "duckdb/common/types/data_chunk.hpp" #include "duckdb/common/winapi.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/main/client_properties.hpp" namespace duckdb { @@ -25,7 +25,7 @@ class BaseQueryResult { DUCKDB_API BaseQueryResult(QueryResultType type, StatementType statement_type, StatementProperties properties, vector types, vector names); //! Creates an unsuccessful query result with error condition - DUCKDB_API BaseQueryResult(QueryResultType type, PreservedError error); + DUCKDB_API BaseQueryResult(QueryResultType type, ErrorData error); DUCKDB_API virtual ~BaseQueryResult(); //! The type of the result (MATERIALIZED or STREAMING) @@ -41,18 +41,18 @@ class BaseQueryResult { public: [[noreturn]] DUCKDB_API void ThrowError(const string &prepended_message = "") const; - DUCKDB_API void SetError(PreservedError error); + DUCKDB_API void SetError(ErrorData error); DUCKDB_API bool HasError() const; DUCKDB_API const ExceptionType &GetErrorType() const; DUCKDB_API const std::string &GetError(); - DUCKDB_API PreservedError &GetErrorObject(); + DUCKDB_API ErrorData &GetErrorObject(); DUCKDB_API idx_t ColumnCount(); protected: //! Whether or not execution was successful bool success; //! The error (in case execution was not successful) - PreservedError error; + ErrorData error; }; //! The QueryResult object holds the result of a query. It can either be a MaterializedQueryResult, in which case the @@ -64,7 +64,7 @@ class QueryResult : public BaseQueryResult { DUCKDB_API QueryResult(QueryResultType type, StatementType statement_type, StatementProperties properties, vector types, vector names, ClientProperties client_properties); //! Creates an unsuccessful query result with error condition - DUCKDB_API QueryResult(QueryResultType type, PreservedError error); + DUCKDB_API QueryResult(QueryResultType type, ErrorData error); DUCKDB_API virtual ~QueryResult() override; //! Properties from the client context @@ -108,18 +108,15 @@ class QueryResult : public BaseQueryResult { //! Fetch() until both results are exhausted. The data in the results will be lost. DUCKDB_API bool Equals(QueryResult &other); - bool TryFetch(unique_ptr &result, PreservedError &error) { + bool TryFetch(unique_ptr &result, ErrorData &error) { try { result = Fetch(); return success; - } catch (const Exception &ex) { - error = PreservedError(ex); - return false; } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); return false; } catch (...) { - error = PreservedError("Unknown error in Fetch"); + error = ErrorData("Unknown error in Fetch"); return false; } } 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 cbd60ae91..0590cf6a9 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 @@ -8,7 +8,7 @@ #pragma once -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "duckdb/main/relation/table_function_relation.hpp" #include "duckdb/common/shared_ptr.hpp" #include "duckdb/common/case_insensitive_map.hpp" diff --git a/src/duckdb/src/include/duckdb/main/secret/secret.hpp b/src/duckdb/src/include/duckdb/main/secret/secret.hpp new file mode 100644 index 000000000..de5753d2d --- /dev/null +++ b/src/duckdb/src/include/duckdb/main/secret/secret.hpp @@ -0,0 +1,197 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/main/secret/secret.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/common.hpp" +#include "duckdb/common/named_parameter_map.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" + +namespace duckdb { +class BaseSecret; + +//! Whether a secret is persistent or temporary +enum class SecretPersistType : uint8_t { DEFAULT, TEMPORARY, PERSISTENT }; + +//! Input passed to a CreateSecretFunction +struct CreateSecretInput { + //! type + string type; + //! mode + string provider; + //! should the secret be persisted? + string storage_type; + //! (optional) alias provided by user + string name; + //! (optional) scope provided by user + vector scope; + //! (optional) named parameter map, each create secret function has defined it's own set of these + case_insensitive_map_t options; +}; + +typedef unique_ptr (*secret_deserializer_t)(Deserializer &deserializer, BaseSecret base_secret); +typedef unique_ptr (*create_secret_function_t)(ClientContext &context, CreateSecretInput &input); + +//! A CreateSecretFunction is a function adds a provider for a secret type. +class CreateSecretFunction { +public: + string secret_type; + string provider; + create_secret_function_t function; + named_parameter_type_map_t named_parameters; +}; + +//! CreateSecretFunctionsSet contains multiple functions of a single type, identified by the provider. The provider +//! should be seen as the method of secret creation. (e.g. user-provided config, env variables, auto-detect) +class CreateSecretFunctionSet { +public: + CreateSecretFunctionSet(string &name) : name(name) {}; + bool ProviderExists(const string &provider_name); + void AddFunction(CreateSecretFunction &function, OnCreateConflict on_conflict); + CreateSecretFunction &GetFunction(const string &provider); + +protected: + //! Create Secret Function type name + string name; + //! Maps of provider -> function + case_insensitive_map_t functions; +}; + +//! Determines whether the secrets are allowed to be shown +enum class SecretDisplayType : uint8_t { REDACTED, UNREDACTED }; + +//! Secret types contain the base settings of a secret +struct SecretType { + //! Unique name identifying the secret type + string name; + //! The deserialization function for the type + secret_deserializer_t deserializer; + //! Provider to use when non is specified + string default_provider; +}; + +//! Base class from which BaseSecret classes can be made. +class BaseSecret { + friend class SecretManager; + +public: + BaseSecret(const vector &prefix_paths, const string &type, const string &provider, const string &name) + : prefix_paths(prefix_paths), type(type), provider(provider), name(name), serializable(false) { + D_ASSERT(!type.empty()); + } + BaseSecret(const BaseSecret &other) + : prefix_paths(other.prefix_paths), type(other.type), provider(other.provider), name(other.name), + serializable(other.serializable) { + D_ASSERT(!type.empty()); + } + virtual ~BaseSecret() = default; + + //! The score of how well this secret's scope matches the path (by default: the length of the longest matching + //! prefix) + virtual int64_t MatchScore(const string &path) const; + //! Prints the secret as a string + virtual string ToString(SecretDisplayType mode = SecretDisplayType::REDACTED) const; + //! Serialize this secret + virtual void Serialize(Serializer &serializer) const; + + //! Getters + const vector &GetScope() const { + return prefix_paths; + } + const string &GetType() const { + return type; + } + const string &GetProvider() const { + return provider; + } + const string &GetName() const { + return name; + } + bool IsSerializable() const { + return serializable; + } + +protected: + //! Helper function to serialize the base BaseSecret class variables + virtual void SerializeBaseSecret(Serializer &serializer) const final; + + //! prefixes to which the secret applies + vector prefix_paths; + + //! Type of secret + string type; + //! Provider of the secret + string provider; + //! Name of the secret + string name; + //! Whether the secret can be serialized/deserialized + bool serializable; +}; + +//! The KeyValueSecret is a class that implements a Secret as a set of key -> values. This class can be used +//! for most use-cases of secrets as secrets generally tend to fit in a key value map. +class KeyValueSecret : public BaseSecret { +public: + KeyValueSecret(const vector &prefix_paths, const string &type, const string &provider, const string &name) + : BaseSecret(prefix_paths, type, provider, name) { + D_ASSERT(!type.empty()); + serializable = true; + } + KeyValueSecret(BaseSecret &secret) + : BaseSecret(secret.GetScope(), secret.GetType(), secret.GetProvider(), secret.GetName()) { + serializable = true; + }; + KeyValueSecret(const KeyValueSecret &secret) + : BaseSecret(secret.GetScope(), secret.GetType(), secret.GetProvider(), secret.GetName()) { + secret_map = secret.secret_map; + redact_keys = secret.redact_keys; + serializable = true; + }; + KeyValueSecret(KeyValueSecret &&secret) + : BaseSecret(secret.GetScope(), secret.GetType(), secret.GetProvider(), secret.GetName()) { + secret_map = std::move(secret.secret_map); + redact_keys = std::move(secret.redact_keys); + serializable = true; + }; + + //! Print the secret as a key value map in the format 'key1=value;key2=value2' + virtual string ToString(SecretDisplayType mode = SecretDisplayType::REDACTED) const override; + void Serialize(Serializer &serializer) const override; + + //! Tries to get the value at key , depending on error_on_missing will throw or return Value() + Value TryGetValue(const string &key, bool error_on_missing = false) const; + + // FIXME: use serialization scripts + template + static unique_ptr Deserialize(Deserializer &deserializer, BaseSecret base_secret) { + auto result = make_uniq(base_secret); + Value secret_map_value; + deserializer.ReadProperty(201, "secret_map", secret_map_value); + + for (const auto &entry : ListValue::GetChildren(secret_map_value)) { + auto kv_struct = StructValue::GetChildren(entry); + result->secret_map[kv_struct[0].ToString()] = kv_struct[1].ToString(); + } + + Value redact_set_value; + deserializer.ReadProperty(202, "redact_keys", redact_set_value); + for (const auto &entry : ListValue::GetChildren(redact_set_value)) { + result->redact_keys.insert(entry.ToString()); + } + + return duckdb::unique_ptr_cast(std::move(result)); + } + + //! the map of key -> values that make up the secret + case_insensitive_tree_t secret_map; + //! keys that are sensitive and should be redacted + case_insensitive_set_t redact_keys; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/secret/secret_manager.hpp b/src/duckdb/src/include/duckdb/main/secret/secret_manager.hpp new file mode 100644 index 000000000..aa7676bc1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/main/secret/secret_manager.hpp @@ -0,0 +1,194 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/main/secret/duck_secret_manager.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/catalog/default/default_generator.hpp" +#include "duckdb/common/common.hpp" +#include "duckdb/main/secret/secret.hpp" +#include "duckdb/main/secret/secret_manager.hpp" +#include "duckdb/main/secret/secret_storage.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" + +namespace duckdb { +class SecretManager; +struct DBConfig; +class SchemaCatalogEntry; + +//! Return value of a Secret Lookup +struct SecretMatch { +public: + SecretMatch() : secret_entry(nullptr), score(NumericLimits::Minimum()) { + } + SecretMatch(SecretEntry &secret_entry, int64_t score) : secret_entry(&secret_entry), score(score) { + } + + //! Get the secret + const BaseSecret &GetSecret(); + + bool HasMatch() { + return secret_entry; + } + + optional_ptr secret_entry; + int64_t score; +}; + +//! A Secret Entry in the secret manager +struct SecretEntry { +public: + SecretEntry(unique_ptr secret) : secret(std::move(secret)) {}; + + //! Whether the secret is persistent + SecretPersistType persist_type; + //! The storage backend of the secret + string storage_mode; + //! The secret pointer + unique_ptr secret; +}; + +struct SecretManagerConfig { + static constexpr const bool DEFAULT_ALLOW_PERSISTENT_SECRETS = true; + //! The default persistence type for secrets + SecretPersistType default_persist_type = SecretPersistType::TEMPORARY; + //! Secret Path can be changed by until the secret manager is initialized, after that it will be set automatically + string secret_path = ""; + //! The default secret path is determined on startup and can be used to reset the secret path + string default_secret_path = ""; + //! The storage type for persistent secrets when none is specified; + string default_persistent_storage = ""; + //! Persistent secrets are enabled by default + bool allow_persistent_secrets = DEFAULT_ALLOW_PERSISTENT_SECRETS; +}; + +//! The Secret Manager for DuckDB. Can handle both temporary and persistent secrets +class SecretManager { + friend struct SecretEntry; + +public: + explicit SecretManager() = default; + virtual ~SecretManager() = default; + + //! The default storage backends + static constexpr const char *TEMPORARY_STORAGE_NAME = "memory"; + static constexpr const char *LOCAL_FILE_STORAGE_NAME = "local_file"; + + //! Static Helper Functions + DUCKDB_API static SecretManager &Get(ClientContext &context); + DUCKDB_API static SecretManager &Get(DatabaseInstance &db); + + // Initialize the secret manager with the DB instance + DUCKDB_API void Initialize(DatabaseInstance &db); + //! Load a secret storage + DUCKDB_API void LoadSecretStorage(unique_ptr storage); + + //! Deserialize a secret by automatically selecting the correct deserializer + DUCKDB_API unique_ptr DeserializeSecret(Deserializer &deserializer); + //! Register a new SecretType + DUCKDB_API void RegisterSecretType(SecretType &type); + //! Lookup a SecretType + DUCKDB_API SecretType LookupType(const string &type); + //! Register a Secret Function i.e. a secret provider for a secret type + DUCKDB_API void RegisterSecretFunction(CreateSecretFunction function, OnCreateConflict on_conflict); + //! Register a secret by providing a secret manually + DUCKDB_API optional_ptr RegisterSecret(CatalogTransaction transaction, + unique_ptr secret, + OnCreateConflict on_conflict, SecretPersistType persist_type, + const string &storage = ""); + //! Create a secret from a CreateSecretInfo + DUCKDB_API optional_ptr CreateSecret(ClientContext &context, const CreateSecretInfo &info); + //! The Bind for create secret is done by the secret manager + DUCKDB_API BoundStatement BindCreateSecret(CatalogTransaction transaction, CreateSecretInfo &info); + //! Lookup the best matching secret by matching the secret scopes to the path + DUCKDB_API SecretMatch LookupSecret(CatalogTransaction transaction, const string &path, const string &type); + //! Get a secret by name, optionally from a specific storage + DUCKDB_API optional_ptr GetSecretByName(CatalogTransaction transaction, const string &name, + const string &storage = ""); + //! Delete a secret by name, optionally by providing the storage to drop from + DUCKDB_API void DropSecretByName(CatalogTransaction transaction, const string &name, + OnEntryNotFound on_entry_not_found, + SecretPersistType persist_type = SecretPersistType::DEFAULT, + const string &storage = ""); + //! List all secrets from all secret storages + DUCKDB_API vector> AllSecrets(CatalogTransaction transaction); + + //! Secret Manager settings + DUCKDB_API virtual void SetEnablePersistentSecrets(bool enabled); + DUCKDB_API virtual void ResetEnablePersistentSecrets(); + DUCKDB_API virtual bool PersistentSecretsEnabled(); + + DUCKDB_API virtual void SetDefaultStorage(const string &storage); + DUCKDB_API virtual void ResetDefaultStorage(); + DUCKDB_API virtual string DefaultStorage(); + + DUCKDB_API virtual void SetPersistentSecretPath(const string &path); + DUCKDB_API virtual void ResetPersistentSecretPath(); + DUCKDB_API virtual string PersistentSecretPath(); + + //! Utility functions + DUCKDB_API void DropSecretByName(ClientContext &context, const string &name, OnEntryNotFound on_entry_not_found, + SecretPersistType persist_type = SecretPersistType::DEFAULT, + const string &storage = ""); + +private: + //! Lookup a SecretType + SecretType LookupTypeInternal(const string &type); + //! Lookup a CreateSecretFunction + optional_ptr LookupFunctionInternal(const string &type, const string &provider); + //! Register a new Secret + optional_ptr RegisterSecretInternal(CatalogTransaction transaction, + unique_ptr secret, OnCreateConflict on_conflict, + SecretPersistType persist_type, const string &storage = ""); + //! Initialize the secret catalog_set and persistent secrets (lazily) + void InitializeSecrets(CatalogTransaction transaction); + //! Load a secret storage + void LoadSecretStorageInternal(unique_ptr storage); + + //! Autoload extension for specific secret type + void AutoloadExtensionForType(const string &type); + //! Autoload extension for specific secret function + void AutoloadExtensionForFunction(const string &type, const string &provider); + + //! Thread-safe accessors for secret_storages + vector> GetSecretStorages(); + optional_ptr GetSecretStorage(const string &name); + + //! Throw an exception if the secret manager is initialized + void ThrowOnSettingChangeIfInitialized(); + + //! Lock for types, functions, settings and storages + mutex manager_lock; + //! Secret functions; + case_insensitive_map_t secret_functions; + //! Secret types; + case_insensitive_map_t secret_types; + //! Map of all registered SecretStorages + case_insensitive_map_t> secret_storages; + //! While false, secret manager settings can still be changed + atomic initialized {false}; + //! Configuration for secret manager + SecretManagerConfig config; + //! Pointer to current db instance + optional_ptr db; +}; + +//! The DefaultGenerator for persistent secrets. This is used to store lazy loaded secrets in the catalog +class DefaultSecretGenerator : public DefaultGenerator { +public: + DefaultSecretGenerator(Catalog &catalog, SecretManager &secret_manager, case_insensitive_set_t &persistent_secrets); + +public: + unique_ptr CreateDefaultEntry(ClientContext &context, const string &entry_name) override; + vector GetDefaultEntries() override; + +protected: + SecretManager &secret_manager; + case_insensitive_set_t persistent_secrets; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/secret/secret_storage.hpp b/src/duckdb/src/include/duckdb/main/secret/secret_storage.hpp new file mode 100644 index 000000000..25dfdbf8d --- /dev/null +++ b/src/duckdb/src/include/duckdb/main/secret/secret_storage.hpp @@ -0,0 +1,164 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/main/secret/secret_storage.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/common.hpp" +#include "duckdb/common/enums/on_entry_not_found.hpp" + +namespace duckdb { + +class BaseSecret; +class Catalog; +class CatalogSet; +struct CatalogTransaction; +class DatabaseInstance; +struct SecretMatch; +struct SecretEntry; + +//! Base class for SecretStorage API +class SecretStorage { + friend class SecretManager; + +public: + SecretStorage(const string &name) : storage_name(name), persistent(false) {}; + virtual ~SecretStorage() = default; + +public: + //! SecretStorage API + + //! Get the storage name (e.g. local_file, :memory:) + virtual string &GetName() { + return storage_name; + }; + + //! Store a secret + virtual optional_ptr StoreSecret(unique_ptr secret, OnCreateConflict on_conflict, + optional_ptr transaction = nullptr) = 0; + //! Get all secrets + virtual vector> AllSecrets(optional_ptr transaction = nullptr) = 0; + //! Drop secret by name + virtual void DropSecretByName(const string &name, OnEntryNotFound on_entry_not_found, + optional_ptr transaction = nullptr) = 0; + //! Get best match + virtual SecretMatch LookupSecret(const string &path, const string &type, + optional_ptr transaction = nullptr) = 0; + //! Get a secret by name + virtual optional_ptr GetSecretByName(const string &name, + optional_ptr transaction = nullptr) = 0; + + //! Return the offset associated to this storage for tie-breaking secrets between storages + virtual int64_t GetTieBreakOffset() = 0; + + //! Returns include_in_lookups, used to create secret storage + virtual bool IncludeInLookups() { + return true; + } + + virtual bool Persistent() const { + return persistent; + } + +protected: + //! Helper function to select the best matching secret within a storage. Tie-breaks within a storage are broken + //! by secret name by default. + SecretMatch SelectBestMatch(SecretEntry &secret_entry, const string &path, SecretMatch ¤t_best); + + //! Offsets the score to tie-break secrets giving preference to the storage with the lowest storage_penalty + //! the base implementation will be chosen last in a tie-break + int64_t OffsetMatchScore(int64_t score) { + return 100 * score - GetTieBreakOffset(); + } + + //! Name of the storage backend (e.g. temporary, file, etc) + string storage_name; + //! Whether entries in this storage will survive duckdb reboots + bool persistent; +}; + +//! Wrapper struct around a SecretEntry to allow storing it +struct SecretCatalogEntry : public InCatalogEntry { +public: + SecretCatalogEntry(unique_ptr secret_p, Catalog &catalog); + SecretCatalogEntry(unique_ptr secret_p, Catalog &catalog); + + //! The secret to store in a catalog + unique_ptr secret; +}; + +//! Base Implementation for catalog set based secret storage +class CatalogSetSecretStorage : public SecretStorage { +public: + CatalogSetSecretStorage(DatabaseInstance &db_instance, const string &name_p) + : SecretStorage(name_p), db(db_instance) {}; + +public: + //! SecretStorage API + string &GetName() override { + return storage_name; + }; + + virtual optional_ptr StoreSecret(unique_ptr secret, OnCreateConflict on_conflict, + optional_ptr transaction = nullptr) override; + vector> AllSecrets(optional_ptr transaction = nullptr) override; + void DropSecretByName(const string &name, OnEntryNotFound on_entry_not_found, + optional_ptr transaction = nullptr) override; + SecretMatch LookupSecret(const string &path, const string &type, + optional_ptr transaction = nullptr) override; + optional_ptr GetSecretByName(const string &name, + optional_ptr transaction = nullptr) override; + +protected: + //! Callback called on Store to allow child classes to implement persistence. + virtual void WriteSecret(const BaseSecret &secret, OnCreateConflict on_conflict); + virtual void RemoveSecret(const string &name, OnEntryNotFound on_entry_not_found); + //! Returns the CatalogTransaction in `transaction` if not set, return the System transaction + CatalogTransaction GetTransactionOrDefault(optional_ptr transaction); + + //! CatalogSet containing the secrets + unique_ptr secrets; + //! DB instance for accessing the system catalog transaction + DatabaseInstance &db; +}; + +class TemporarySecretStorage : public CatalogSetSecretStorage { +public: + TemporarySecretStorage(const string &name_p, DatabaseInstance &db_p) : CatalogSetSecretStorage(db_p, name_p) { + secrets = make_uniq(Catalog::GetSystemCatalog(db)); + persistent = false; + } + + int64_t GetTieBreakOffset() override { + return 10; + } + +protected: +}; + +class LocalFileSecretStorage : public CatalogSetSecretStorage { +public: + LocalFileSecretStorage(SecretManager &manager, DatabaseInstance &db, const string &name_p, + const string &secret_path); + + int64_t GetTieBreakOffset() override { + return 20; + } + +protected: + //! Implements the writes to disk + void WriteSecret(const BaseSecret &secret, OnCreateConflict on_conflict) override; + //! Implements the deletes from disk + virtual void RemoveSecret(const string &secret, OnEntryNotFound on_entry_not_found) override; + + //! Set of persistent secrets that are lazily loaded + case_insensitive_set_t persistent_secrets; + //! Path that is searched for secrets; + string secret_path; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/main/settings.hpp b/src/duckdb/src/include/duckdb/main/settings.hpp index f0e92267c..72433204b 100644 --- a/src/duckdb/src/include/duckdb/main/settings.hpp +++ b/src/duckdb/src/include/duckdb/main/settings.hpp @@ -16,6 +16,8 @@ class ClientContext; class DatabaseInstance; struct DBConfig; +const string GetDefaultUserAgent(); + struct AccessModeSetting { static constexpr const char *Name = "access_mode"; static constexpr const char *Description = "Access mode of the database (AUTOMATIC, READ_ONLY or READ_WRITE)"; @@ -25,6 +27,16 @@ struct AccessModeSetting { static Value GetSetting(ClientContext &context); }; +struct AllowPersistentSecrets { + static constexpr const char *Name = "allow_persistent_secrets"; + static constexpr const char *Description = + "Allow the creation of persistent secrets, that are stored and loaded on restarts"; + static constexpr const LogicalTypeId InputType = LogicalTypeId::BOOLEAN; + static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); + static void ResetGlobal(DatabaseInstance *db, DBConfig &config); + static Value GetSetting(ClientContext &context); +}; + struct CheckpointThresholdSetting { static constexpr const char *Name = "checkpoint_threshold"; static constexpr const char *Description = @@ -131,6 +143,15 @@ struct DefaultNullOrderSetting { static Value GetSetting(ClientContext &context); }; +struct DefaultSecretStorage { + static constexpr const char *Name = "default_secret_storage"; + static constexpr const char *Description = "Allows switching the default storage for secrets"; + 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 DisabledFileSystemsSetting { static constexpr const char *Name = "disabled_filesystems"; static constexpr const char *Description = "Disable specific file systems preventing access (e.g. LocalFileSystem)"; @@ -183,8 +204,8 @@ struct CustomExtensionRepository { static constexpr const char *Name = "custom_extension_repository"; static constexpr const char *Description = "Overrides the custom endpoint for remote extension installation"; static constexpr const LogicalTypeId InputType = LogicalTypeId::VARCHAR; - static void SetLocal(ClientContext &context, const Value ¶meter); - static void ResetLocal(ClientContext &context); + static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); + static void ResetGlobal(DatabaseInstance *db, DBConfig &config); static Value GetSetting(ClientContext &context); }; @@ -193,8 +214,8 @@ struct AutoloadExtensionRepository { static constexpr const char *Description = "Overrides the custom endpoint for extension installation on autoloading"; static constexpr const LogicalTypeId InputType = LogicalTypeId::VARCHAR; - static void SetLocal(ClientContext &context, const Value ¶meter); - static void ResetLocal(ClientContext &context); + static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); + static void ResetGlobal(DatabaseInstance *db, DBConfig &config); static Value GetSetting(ClientContext &context); }; @@ -266,6 +287,15 @@ struct EnableProgressBarPrintSetting { static Value GetSetting(ClientContext &context); }; +struct ErrorsAsJsonSetting { + static constexpr const char *Name = "errors_as_json"; + static constexpr const char *Description = "Output error messages as structured JSON instead of as a raw string"; + static constexpr const LogicalTypeId InputType = LogicalTypeId::BOOLEAN; + static void SetLocal(ClientContext &context, const Value ¶meter); + static void ResetLocal(ClientContext &context); + static Value GetSetting(ClientContext &context); +}; + struct ExplainOutputSetting { static constexpr const char *Name = "explain_output"; static constexpr const char *Description = "Output of EXPLAIN statements (ALL, OPTIMIZED_ONLY, PHYSICAL_ONLY)"; @@ -388,6 +418,15 @@ struct MaximumMemorySetting { static Value GetSetting(ClientContext &context); }; +struct OldImplicitCasting { + static constexpr const char *Name = "old_implicit_casting"; + static constexpr const char *Description = "Allow implicit casting to/from VARCHAR"; + static constexpr const LogicalTypeId InputType = LogicalTypeId::BOOLEAN; + static void SetGlobal(DatabaseInstance *db, DBConfig &config, const Value ¶meter); + static void ResetGlobal(DatabaseInstance *db, DBConfig &config); + static Value GetSetting(ClientContext &context); +}; + struct PasswordSetting { static constexpr const char *Name = "password"; static constexpr const char *Description = "The password to use. Ignored for legacy compatibility."; @@ -515,6 +554,15 @@ struct SearchPathSetting { static Value GetSetting(ClientContext &context); }; +struct SecretDirectorySetting { + static constexpr const char *Name = "secret_directory"; + static constexpr const char *Description = "Set the directory to which persistent secrets are stored"; + 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 TempDirectorySetting { static constexpr const char *Name = "temp_directory"; static constexpr const char *Description = "Set the directory to which to write temp files"; diff --git a/src/duckdb/src/include/duckdb/optimizer/deliminator.hpp b/src/duckdb/src/include/duckdb/optimizer/deliminator.hpp index 482c4064a..fb6d1d913 100644 --- a/src/duckdb/src/include/duckdb/optimizer/deliminator.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/deliminator.hpp @@ -25,13 +25,16 @@ class Deliminator { private: //! Finds DelimJoins and their corresponding DelimGets void FindCandidates(unique_ptr &op, vector &candidates); - void FindJoinWithDelimGet(unique_ptr &op, DelimCandidate &candidate); + void FindJoinWithDelimGet(unique_ptr &op, DelimCandidate &candidate, idx_t depth = 0); + //! Whether the DelimJoin is selective + bool HasSelection(const LogicalOperator &delim_join); //! Remove joins with a DelimGet bool RemoveJoinWithDelimGet(LogicalComparisonJoin &delim_join, const idx_t delim_get_count, unique_ptr &join, bool &all_equality_conditions); bool RemoveInequalityJoinWithDelimGet(LogicalComparisonJoin &delim_join, const idx_t delim_get_count, unique_ptr &join, const vector &replacement_bindings); + void TrySwitchSingleToLeft(LogicalComparisonJoin &delim_join); private: optional_ptr root; diff --git a/src/duckdb/src/include/duckdb/optimizer/filter_pushdown.hpp b/src/duckdb/src/include/duckdb/optimizer/filter_pushdown.hpp index ab6f04b06..70ba4fdf3 100644 --- a/src/duckdb/src/include/duckdb/optimizer/filter_pushdown.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/filter_pushdown.hpp @@ -65,6 +65,8 @@ class FilterPushdown { // Pushdown a left join unique_ptr PushdownLeftJoin(unique_ptr op, unordered_set &left_bindings, unordered_set &right_bindings); + + unique_ptr PushdownSemiAntiJoin(unique_ptr op); // Pushdown a mark join unique_ptr PushdownMarkJoin(unique_ptr op, unordered_set &left_bindings, unordered_set &right_bindings); diff --git a/src/duckdb/src/include/duckdb/optimizer/join_order/query_graph_manager.hpp b/src/duckdb/src/include/duckdb/optimizer/join_order/query_graph_manager.hpp index 4e5e08add..79aa37fb9 100644 --- a/src/duckdb/src/include/duckdb/optimizer/join_order/query_graph_manager.hpp +++ b/src/duckdb/src/include/duckdb/optimizer/join_order/query_graph_manager.hpp @@ -90,7 +90,7 @@ class QueryGraphManager { //! (Basically we put lower expected cardinality columns on the build side, and larger //! tables on the probe side) unique_ptr LeftRightOptimizations(unique_ptr op); - void TryFlipChildren(LogicalOperator &op, JoinType inverse, idx_t cardinality_ratio = 1); + void TryFlipChildren(LogicalOperator &op, idx_t cardinality_ratio = 1); private: vector> filter_operators; diff --git a/src/duckdb/src/include/duckdb/parallel/event.hpp b/src/duckdb/src/include/duckdb/parallel/event.hpp index 1cfee6917..354012020 100644 --- a/src/duckdb/src/include/duckdb/parallel/event.hpp +++ b/src/duckdb/src/include/duckdb/parallel/event.hpp @@ -37,7 +37,7 @@ class Event : public std::enable_shared_from_this { bool HasDependencies() const { return total_dependencies != 0; } - const vector &GetParentsVerification() const; + const vector> &GetParentsVerification() const; void CompleteDependency(); @@ -79,7 +79,7 @@ class Event : public std::enable_shared_from_this { //! The events that depend on this event to run vector> parents; //! Raw pointers to the parents (used for verification only) - vector parents_raw; + vector> parents_raw; //! Whether or not the event is finished executing atomic finished; diff --git a/src/duckdb/src/include/duckdb/parallel/meta_pipeline.hpp b/src/duckdb/src/include/duckdb/parallel/meta_pipeline.hpp index 82b92ba51..5bf58ef80 100644 --- a/src/duckdb/src/include/duckdb/parallel/meta_pipeline.hpp +++ b/src/duckdb/src/include/duckdb/parallel/meta_pipeline.hpp @@ -8,19 +8,11 @@ #pragma once +#include "duckdb/common/reference_map.hpp" #include "duckdb/execution/physical_operator.hpp" namespace duckdb { -class PhysicalRecursiveCTE; - -struct PipelineFinishGroup { - explicit PipelineFinishGroup(Pipeline *group_base_p) : group_base(group_base_p) { - } - Pipeline *group_base; - unordered_set group_members; -}; - //! MetaPipeline represents a set of pipelines that all have the same sink class MetaPipeline : public std::enable_shared_from_this { //! We follow these rules when building: @@ -34,7 +26,7 @@ class MetaPipeline : public std::enable_shared_from_this { //! * And all pipelines that were added to the MetaPipeline after 'current' public: //! Create a MetaPipeline with the given sink - explicit MetaPipeline(Executor &executor, PipelineBuildState &state, PhysicalOperator *sink); + MetaPipeline(Executor &executor, PipelineBuildState &state, optional_ptr sink); public: //! Get the Executor for this MetaPipeline @@ -51,22 +43,22 @@ class MetaPipeline : public std::enable_shared_from_this { //! Get the MetaPipeline children of this MetaPipeline void GetMetaPipelines(vector> &result, bool recursive, bool skip); //! Get the dependencies (within this MetaPipeline) of the given Pipeline - const vector *GetDependencies(Pipeline *dependant) const; + optional_ptr>> GetDependencies(Pipeline &dependant) const; //! Whether this MetaPipeline has a recursive CTE bool HasRecursiveCTE() const; //! Set the flag that this MetaPipeline is a recursive CTE pipeline void SetRecursiveCTE(); //! Assign a batch index to the given pipeline - void AssignNextBatchIndex(Pipeline *pipeline); + void AssignNextBatchIndex(Pipeline &pipeline); //! Let 'dependant' depend on all pipeline that were created since 'start', //! where 'including' determines whether 'start' is added to the dependencies - void AddDependenciesFrom(Pipeline *dependant, Pipeline *start, bool including); + void AddDependenciesFrom(Pipeline &dependant, Pipeline &start, bool including); //! Make sure that the given pipeline has its own PipelineFinishEvent (e.g., for IEJoin - double Finalize) - void AddFinishEvent(Pipeline *pipeline); + void AddFinishEvent(Pipeline &pipeline); //! Whether the pipeline needs its own PipelineFinishEvent - bool HasFinishEvent(Pipeline *pipeline) const; + bool HasFinishEvent(Pipeline &pipeline) const; //! Whether this pipeline is part of a PipelineFinishEvent - optional_ptr GetFinishGroup(Pipeline *pipeline) const; + optional_ptr GetFinishGroup(Pipeline &pipeline) const; public: //! Build the MetaPipeline with 'op' as the first operator (excl. the shared sink) @@ -75,12 +67,12 @@ class MetaPipeline : public std::enable_shared_from_this { void Ready(); //! Create an empty pipeline within this MetaPipeline - Pipeline *CreatePipeline(); + Pipeline &CreatePipeline(); //! Create a union pipeline (clone of 'current') - Pipeline *CreateUnionPipeline(Pipeline ¤t, bool order_matters); + Pipeline &CreateUnionPipeline(Pipeline ¤t, bool order_matters); //! Create a child pipeline op 'current' starting at 'op', //! where 'last_pipeline' is the last pipeline added before building out 'current' - void CreateChildPipeline(Pipeline ¤t, PhysicalOperator &op, Pipeline *last_pipeline); + void CreateChildPipeline(Pipeline ¤t, PhysicalOperator &op, Pipeline &last_pipeline); //! Create a MetaPipeline child that 'current' depends on MetaPipeline &CreateChildMetaPipeline(Pipeline ¤t, PhysicalOperator &op); @@ -96,15 +88,15 @@ class MetaPipeline : public std::enable_shared_from_this { //! All pipelines with a different source, but the same sink vector> pipelines; //! Dependencies within this MetaPipeline - unordered_map> dependencies; + reference_map_t>> dependencies; //! Other MetaPipelines that this MetaPipeline depends on vector> children; //! Next batch index idx_t next_batch_index; //! Pipelines (other than the base pipeline) that need their own PipelineFinishEvent (e.g., for IEJoin) - unordered_set finish_pipelines; + reference_set_t finish_pipelines; //! Mapping from pipeline (e.g., child or union) to finish pipeline - unordered_map finish_map; + reference_map_t finish_map; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/base_expression.hpp b/src/duckdb/src/include/duckdb/parser/base_expression.hpp index 08c3e1dbd..aed84f5ee 100644 --- a/src/duckdb/src/include/duckdb/parser/base_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/base_expression.hpp @@ -11,6 +11,7 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/enums/expression_type.hpp" #include "duckdb/common/exception.hpp" +#include "duckdb/common/optional_idx.hpp" namespace duckdb { @@ -40,6 +41,8 @@ class BaseExpression { ExpressionClass expression_class; //! The alias of the expression, string alias; + //! The location in the query (if any) + optional_idx query_location; public: //! Returns true if this expression is an aggregate or not. diff --git a/src/duckdb/src/include/duckdb/parser/column_definition.hpp b/src/duckdb/src/include/duckdb/parser/column_definition.hpp index 9c0ae77f4..79d1110f6 100644 --- a/src/duckdb/src/include/duckdb/parser/column_definition.hpp +++ b/src/duckdb/src/include/duckdb/parser/column_definition.hpp @@ -44,6 +44,10 @@ class ColumnDefinition { DUCKDB_API const string &Name() const; void SetName(const string &name); + //! comment + DUCKDB_API const Value &Comment() const; + void SetComment(const Value &comment); + //! compression_type const duckdb::CompressionType &CompressionType() const; void SetCompressionType(duckdb::CompressionType compression_type); @@ -98,6 +102,8 @@ class ColumnDefinition { //! The default value of the column (for non-generated columns) //! The generated column expression (for generated columns) unique_ptr expression; + //! Comment on this column + Value comment; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/expression/lambda_expression.hpp b/src/duckdb/src/include/duckdb/parser/expression/lambda_expression.hpp index 5eae6e942..f3b2c8cc6 100644 --- a/src/duckdb/src/include/duckdb/parser/expression/lambda_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/expression/lambda_expression.hpp @@ -8,15 +8,16 @@ #pragma once -#include "duckdb/parser/parsed_expression.hpp" +#include "duckdb/common/unordered_set.hpp" #include "duckdb/common/vector.hpp" +#include "duckdb/parser/parsed_expression.hpp" namespace duckdb { //! LambdaExpression represents either: -//! 1. A lambda operator that can be used for e.g. mapping an expression to a list -//! 2. An OperatorExpression with the "->" operator -//! Lambda expressions are written in the form of "params -> expr", e.g. "x -> x + 1" +//! 1. A lambda function that can be used for, e.g., mapping an expression to a list +//! 2. An OperatorExpression with the "->" operator (JSON) +//! Lambda expressions are written in the form of "params -> expr", e.g., "x -> x + 1" class LambdaExpression : public ParsedExpression { public: static constexpr const ExpressionClass TYPE = ExpressionClass::LAMBDA; @@ -24,18 +25,24 @@ class LambdaExpression : public ParsedExpression { public: LambdaExpression(unique_ptr lhs, unique_ptr expr); - // we need the context to determine if this is a list of column references or an expression (for JSON) + //! The LHS of a lambda expression or the JSON "->"-operator. We need the context + //! to determine if the LHS is a list of column references (lambda parameters) or an expression (JSON) unique_ptr lhs; - - vector> params; + //! The lambda or JSON expression (RHS) unique_ptr expr; public: - string ToString() const override; + //! Returns a vector to the column references in the LHS expression, and fills the error message, + //! if the LHS is not a valid lambda parameter list + vector> ExtractColumnRefExpressions(string &error_message); + //! Returns the error message for an invalid lambda parameter list + static string InvalidParametersErrorMessage(); + //! Returns true, if the column_name is a lambda parameter name + static bool IsLambdaParameter(const vector> &lambda_params, const string &column_name); + string ToString() const override; static bool Equal(const LambdaExpression &a, const LambdaExpression &b); hash_t Hash() const override; - unique_ptr Copy() const override; void Serialize(Serializer &serializer) const override; diff --git a/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp b/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp index 92408da0a..6fafaa5eb 100644 --- a/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/expression/lambdaref_expression.hpp @@ -9,9 +9,12 @@ #pragma once #include "duckdb/parser/parsed_expression.hpp" +#include "duckdb/common/optional_ptr.hpp" namespace duckdb { +struct DummyBinding; + //! Represents a reference to a lambda parameter class LambdaRefExpression : public ParsedExpression { public: @@ -34,6 +37,10 @@ class LambdaRefExpression : public ParsedExpression { hash_t Hash() const override; unique_ptr Copy() const override; + //! Traverses the lambda_bindings to find a matching binding for the column_name + static unique_ptr FindMatchingBinding(optional_ptr> &lambda_bindings, + const string ¶meter_name); + void Serialize(Serializer &serializer) const override; static unique_ptr Deserialize(Deserializer &deserializer); }; 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 9629b0ffe..42c0eba97 100644 --- a/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/expression/window_expression.hpp @@ -55,6 +55,8 @@ class WindowExpression : public ParsedExpression { unique_ptr filter_expr; //! True to ignore NULL values bool ignore_nulls; + //! Whether or not the aggregate function is distinct, only used for aggregates + bool distinct; //! The window boundaries WindowBoundary start = WindowBoundary::INVALID; WindowBoundary end = WindowBoundary::INVALID; @@ -91,8 +93,11 @@ class WindowExpression : public ParsedExpression { string result = schema.empty() ? function_name : schema + "." + function_name; result += "("; if (entry.children.size()) { - result += StringUtil::Join(entry.children, entry.children.size(), ", ", - [](const unique_ptr &child) { return child->ToString(); }); + // Only one DISTINCT is allowed (on the first argument) + int distincts = entry.distinct ? 0 : 1; + result += StringUtil::Join(entry.children, entry.children.size(), ", ", [&](const unique_ptr &child) { + return (distincts++ ? "" : "DISTINCT ") + child->ToString(); + }); } // Lead/Lag extra arguments if (entry.offset_expr.get()) { diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_info.hpp index 2f7e5a004..11053374f 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_info.hpp @@ -22,7 +22,8 @@ enum class AlterType : uint8_t { ALTER_SEQUENCE = 3, CHANGE_OWNERSHIP = 4, ALTER_SCALAR_FUNCTION = 5, - ALTER_TABLE_FUNCTION = 6 + ALTER_TABLE_FUNCTION = 6, + SET_COMMENT = 7 }; struct AlterEntryData { diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp index 6001e9a33..7fcbac233 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/alter_table_info.hpp @@ -37,6 +37,26 @@ struct ChangeOwnershipInfo : public AlterInfo { unique_ptr Copy() const override; }; +//===--------------------------------------------------------------------===// +// Set Comment +//===--------------------------------------------------------------------===// +struct SetCommentInfo : public AlterInfo { + SetCommentInfo(CatalogType entry_catalog_type, string entry_catalog, string entry_schema, string entry_name, + Value new_comment_value_p, OnEntryNotFound if_not_found); + + CatalogType entry_catalog_type; + Value comment_value; + +public: + CatalogType GetCatalogType() const override; + unique_ptr Copy() const override; + + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); + + explicit SetCommentInfo(); +}; + //===--------------------------------------------------------------------===// // Alter Table //===--------------------------------------------------------------------===// @@ -50,7 +70,8 @@ enum class AlterTableType : uint8_t { SET_DEFAULT = 6, FOREIGN_KEY_CONSTRAINT = 7, SET_NOT_NULL = 8, - DROP_NOT_NULL = 9 + DROP_NOT_NULL = 9, + SET_COLUMN_COMMENT = 10 }; struct AlterTableInfo : public AlterInfo { @@ -66,7 +87,7 @@ struct AlterTableInfo : public AlterInfo { static unique_ptr Deserialize(Deserializer &deserializer); protected: - AlterTableInfo(AlterTableType type); + explicit AlterTableInfo(AlterTableType type); }; //===--------------------------------------------------------------------===// @@ -186,6 +207,30 @@ struct ChangeColumnTypeInfo : public AlterTableInfo { ChangeColumnTypeInfo(); }; +//===--------------------------------------------------------------------===// +// SetColumnCommentInfo +//===--------------------------------------------------------------------===// +struct SetColumnCommentInfo : public AlterTableInfo { + SetColumnCommentInfo(AlterEntryData data, string column_name, Value comment_value); + ~SetColumnCommentInfo() override; + + //! The column name to alter + string column_name; + //! The target type of the column + Value comment; + +public: + unique_ptr Copy() const override; + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); + + string GetColumnName() const override { + return column_name; + }; + + explicit SetColumnCommentInfo(); +}; + //===--------------------------------------------------------------------===// // SetDefaultInfo //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/comment_on_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/comment_on_info.hpp new file mode 100644 index 000000000..c8093be7c --- /dev/null +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/comment_on_info.hpp @@ -0,0 +1,45 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parser/parsed_data/comment_on_info.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/enums/catalog_type.hpp" +#include "duckdb/common/types/value.hpp" +#include "duckdb/parser/parsed_data/parse_info.hpp" +#include "duckdb/parser/qualified_name.hpp" + +namespace duckdb { + +struct CommentOnInfo : public ParseInfo { +public: + static constexpr const ParseInfoType TYPE = ParseInfoType::COMMENT_ON_INFO; + +public: + CommentOnInfo(); + + //! Catalog type to comment on + CatalogType type; + + //! The catalog name of the entry to comment on + string catalog; + //! The schema name of the entry to comment on + string schema; + //! The name of the entry to comment on + string name; + + //! The comment, can be NULL or a string + Value comment; + +public: + unique_ptr Copy() const; + + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); +}; + +} // namespace duckdb 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 b52d37c33..918c6fcbb 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 @@ -12,6 +12,7 @@ #include "duckdb/parser/parsed_data/parse_info.hpp" #include "duckdb/common/enum_util.hpp" #include "duckdb/common/enums/on_create_conflict.hpp" +#include "duckdb/common/types/value.hpp" namespace duckdb { struct AlterInfo; @@ -42,6 +43,8 @@ struct CreateInfo : public ParseInfo { bool internal; //! The SQL string of the CREATE statement string sql; + //! User provided comment + Value comment; public: void Serialize(Serializer &serializer) const override; diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_secret_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_secret_info.hpp new file mode 100644 index 000000000..3d368938e --- /dev/null +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_secret_info.hpp @@ -0,0 +1,47 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parser/parsed_data/create_secret_info.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/main/secret/secret.hpp" +#include "duckdb/common/enums/catalog_type.hpp" +#include "duckdb/parser/column_definition.hpp" +#include "duckdb/parser/parsed_data/parse_info.hpp" +#include "duckdb/parser/parsed_data/create_info.hpp" +#include "duckdb/common/enums/on_entry_not_found.hpp" +#include "duckdb/common/named_parameter_map.hpp" + +namespace duckdb { + +struct CreateSecretInfo : public CreateInfo { +public: + static constexpr const ParseInfoType TYPE = ParseInfoType::CREATE_SECRET_INFO; + +public: + explicit CreateSecretInfo(OnCreateConflict on_conflict, SecretPersistType persist_type); + //! How to handle conflict + OnCreateConflict on_conflict; + //! Whether the secret can be persisted + SecretPersistType persist_type; + //! The type of secret + string type; + //! Which storage to use (empty for default) + string storage_type; + //! (optionally) the provider of the secret credentials + string provider; + //! (optionally) the name of the secret + string name; + //! (optionally) the scope of the secret + vector scope; + //! Named parameter list (if any) + case_insensitive_map_t options; + + unique_ptr Copy() const; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/create_view_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/create_view_info.hpp index c70a273c0..fd13c2b47 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/create_view_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/create_view_info.hpp @@ -37,6 +37,8 @@ struct CreateViewInfo : public CreateInfo { DUCKDB_API static unique_ptr FromSelect(ClientContext &context, unique_ptr info); //! Gets a bound CreateViewInfo object from a CREATE VIEW statement DUCKDB_API static unique_ptr FromCreateView(ClientContext &context, const string &sql); + //! Parse a SELECT statement from a SQL string + DUCKDB_API static unique_ptr ParseSelect(const string &sql); DUCKDB_API void Serialize(Serializer &serializer) const override; DUCKDB_API static unique_ptr Deserialize(Deserializer &deserializer); diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/drop_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/drop_info.hpp index 57322fa55..841690135 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/drop_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/drop_info.hpp @@ -10,9 +10,11 @@ #include "duckdb/common/enums/catalog_type.hpp" #include "duckdb/parser/parsed_data/parse_info.hpp" +#include "duckdb/parser/parsed_data/extra_drop_info.hpp" #include "duckdb/common/enums/on_entry_not_found.hpp" namespace duckdb { +struct ExtraDropInfo; struct DropInfo : public ParseInfo { public: @@ -20,6 +22,7 @@ struct DropInfo : public ParseInfo { public: DropInfo(); + DropInfo(const DropInfo &info); //! The catalog type to drop CatalogType type; @@ -36,9 +39,11 @@ struct DropInfo : public ParseInfo { bool cascade = false; //! Allow dropping of internal system entries bool allow_drop_internal = false; + //! Extra info related to this drop + unique_ptr extra_drop_info; public: - unique_ptr Copy() const; + virtual unique_ptr Copy() const; void Serialize(Serializer &serializer) const override; static unique_ptr Deserialize(Deserializer &deserializer); diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/extra_drop_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/extra_drop_info.hpp new file mode 100644 index 000000000..83a5834ee --- /dev/null +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/extra_drop_info.hpp @@ -0,0 +1,66 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parser/parsed_data/drop_info.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/main/secret/secret.hpp" +#include "duckdb/common/enums/catalog_type.hpp" +#include "duckdb/parser/parsed_data/parse_info.hpp" +#include "duckdb/common/enums/on_entry_not_found.hpp" + +namespace duckdb { + +enum class ExtraDropInfoType : uint8_t { + INVALID = 0, + + SECRET_INFO = 1 +}; + +struct ExtraDropInfo { + explicit ExtraDropInfo(ExtraDropInfoType info_type) : info_type(info_type) { + } + virtual ~ExtraDropInfo() { + } + + ExtraDropInfoType info_type; + +public: + template + TARGET &Cast() { + D_ASSERT(dynamic_cast(this)); + return reinterpret_cast(*this); + } + + template + const TARGET &Cast() const { + D_ASSERT(dynamic_cast(this)); + return reinterpret_cast(*this); + } + virtual unique_ptr Copy() const = 0; + + virtual void Serialize(Serializer &serializer) const; + static unique_ptr Deserialize(Deserializer &deserializer); +}; + +struct ExtraDropSecretInfo : public ExtraDropInfo { + ExtraDropSecretInfo(); + ExtraDropSecretInfo(const ExtraDropSecretInfo &info); + + //! Secret Persistence + SecretPersistType persist_mode; + //! (optional) the name of the storage to drop from + string secret_storage; + +public: + virtual unique_ptr Copy() const override; + + virtual void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/parsed_data/parse_info.hpp b/src/duckdb/src/include/duckdb/parser/parsed_data/parse_info.hpp index 3cacbd7a3..c99297adf 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_data/parse_info.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_data/parse_info.hpp @@ -17,6 +17,7 @@ enum class ParseInfoType : uint8_t { ATTACH_INFO, COPY_INFO, CREATE_INFO, + CREATE_SECRET_INFO, DETACH_INFO, DROP_INFO, BOUND_EXPORT_DATA, @@ -24,7 +25,8 @@ enum class ParseInfoType : uint8_t { PRAGMA_INFO, SHOW_SELECT_INFO, TRANSACTION_INFO, - VACUUM_INFO + VACUUM_INFO, + COMMENT_ON_INFO }; struct ParseInfo { diff --git a/src/duckdb/src/include/duckdb/parser/parsed_expression.hpp b/src/duckdb/src/include/duckdb/parser/parsed_expression.hpp index 148b7ea93..46acfe9e6 100644 --- a/src/duckdb/src/include/duckdb/parser/parsed_expression.hpp +++ b/src/duckdb/src/include/duckdb/parser/parsed_expression.hpp @@ -33,9 +33,6 @@ class ParsedExpression : public BaseExpression { ParsedExpression(ExpressionType type, ExpressionClass expression_class) : BaseExpression(type, expression_class) { } - //! The location in the query (if any) - idx_t query_location = DConstants::INVALID_INDEX; - public: bool IsAggregate() const override; bool IsWindow() const override; @@ -63,6 +60,7 @@ class ParsedExpression : public BaseExpression { type = other.type; expression_class = other.expression_class; alias = other.alias; + query_location = other.query_location; } }; diff --git a/src/duckdb/src/include/duckdb/parser/parser_extension.hpp b/src/duckdb/src/include/duckdb/parser/parser_extension.hpp index 50fec976d..d00b30d95 100644 --- a/src/duckdb/src/include/duckdb/parser/parser_extension.hpp +++ b/src/duckdb/src/include/duckdb/parser/parser_extension.hpp @@ -51,6 +51,8 @@ struct ParserExtensionParseResult { unique_ptr parse_data; //! The error message (if unsuccessful) string error; + //! The error location (if unsuccessful) + optional_idx error_location; }; typedef ParserExtensionParseResult (*parse_function_t)(ParserExtensionInfo *info, const string &query); diff --git a/src/duckdb/src/include/duckdb/parser/qualified_name.hpp b/src/duckdb/src/include/duckdb/parser/qualified_name.hpp index d389cffb7..ee5947806 100644 --- a/src/duckdb/src/include/duckdb/parser/qualified_name.hpp +++ b/src/duckdb/src/include/duckdb/parser/qualified_name.hpp @@ -9,7 +9,7 @@ #pragma once #include "duckdb/common/string.hpp" -#include "duckdb/common/exception.hpp" +#include "duckdb/common/exception/parser_exception.hpp" #include "duckdb/parser/keyword_helper.hpp" #include "duckdb/common/string_util.hpp" diff --git a/src/duckdb/src/include/duckdb/parser/query_error_context.hpp b/src/duckdb/src/include/duckdb/parser/query_error_context.hpp index 79a5dc409..fd512d7e2 100644 --- a/src/duckdb/src/include/duckdb/parser/query_error_context.hpp +++ b/src/duckdb/src/include/duckdb/parser/query_error_context.hpp @@ -11,38 +11,21 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/vector.hpp" #include "duckdb/common/exception_format_value.hpp" -#include "duckdb/common/optional_ptr.hpp" +#include "duckdb/common/optional_idx.hpp" namespace duckdb { -class SQLStatement; class QueryErrorContext { public: - explicit QueryErrorContext(optional_ptr statement_ = nullptr, - idx_t query_location_ = DConstants::INVALID_INDEX) - : statement(statement_), query_location(query_location_) { + explicit QueryErrorContext(optional_idx query_location_ = optional_idx()) : query_location(query_location_) { } - //! The query statement - optional_ptr statement; //! The location in which the error should be thrown - idx_t query_location; + optional_idx query_location; public: - DUCKDB_API static string Format(const string &query, const string &error_message, int error_location); - - DUCKDB_API string FormatErrorRecursive(const string &msg, vector &values); - template - string FormatErrorRecursive(const string &msg, vector &values, T param, Args... params) { - values.push_back(ExceptionFormatValue::CreateFormatValue(param)); - return FormatErrorRecursive(msg, values, params...); - } - - template - string FormatError(const string &msg, Args... params) { - vector values; - return FormatErrorRecursive(msg, values, params...); - } + static string Format(const string &query, const string &error_message, optional_idx error_loc, + bool add_line_indicator = true); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/statement/list.hpp b/src/duckdb/src/include/duckdb/parser/statement/list.hpp index c5196fff4..d9d04e17c 100644 --- a/src/duckdb/src/include/duckdb/parser/statement/list.hpp +++ b/src/duckdb/src/include/duckdb/parser/statement/list.hpp @@ -19,7 +19,6 @@ #include "duckdb/parser/statement/relation_statement.hpp" #include "duckdb/parser/statement/select_statement.hpp" #include "duckdb/parser/statement/set_statement.hpp" -#include "duckdb/parser/statement/show_statement.hpp" #include "duckdb/parser/statement/transaction_statement.hpp" #include "duckdb/parser/statement/update_statement.hpp" #include "duckdb/parser/statement/vacuum_statement.hpp" diff --git a/src/duckdb/src/include/duckdb/parser/statement/show_statement.hpp b/src/duckdb/src/include/duckdb/parser/statement/show_statement.hpp deleted file mode 100644 index ed5021de4..000000000 --- a/src/duckdb/src/include/duckdb/parser/statement/show_statement.hpp +++ /dev/null @@ -1,32 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/parser/statement/show_statement.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/parser/sql_statement.hpp" -#include "duckdb/parser/parsed_data/show_select_info.hpp" - -namespace duckdb { - -class ShowStatement : public SQLStatement { -public: - static constexpr const StatementType TYPE = StatementType::SHOW_STATEMENT; - -public: - ShowStatement(); - - unique_ptr info; - -protected: - ShowStatement(const ShowStatement &other); - -public: - unique_ptr Copy() const override; -}; - -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/tableref.hpp b/src/duckdb/src/include/duckdb/parser/tableref.hpp index ee4b88208..57a7e4854 100644 --- a/src/duckdb/src/include/duckdb/parser/tableref.hpp +++ b/src/duckdb/src/include/duckdb/parser/tableref.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/common.hpp" +#include "duckdb/common/optional_idx.hpp" #include "duckdb/common/enums/tableref_type.hpp" #include "duckdb/parser/parsed_data/sample_options.hpp" @@ -30,7 +31,7 @@ class TableRef { //! Sample options (if any) unique_ptr sample; //! The location in the query (if any) - idx_t query_location = DConstants::INVALID_INDEX; + optional_idx query_location; public: //! Convert the object to a string diff --git a/src/duckdb/src/include/duckdb/parser/tableref/emptytableref.hpp b/src/duckdb/src/include/duckdb/parser/tableref/emptytableref.hpp index 1f4758e43..20483e561 100644 --- a/src/duckdb/src/include/duckdb/parser/tableref/emptytableref.hpp +++ b/src/duckdb/src/include/duckdb/parser/tableref/emptytableref.hpp @@ -14,10 +14,10 @@ namespace duckdb { class EmptyTableRef : public TableRef { public: - static constexpr const TableReferenceType TYPE = TableReferenceType::EMPTY; + static constexpr const TableReferenceType TYPE = TableReferenceType::EMPTY_FROM; public: - EmptyTableRef() : TableRef(TableReferenceType::EMPTY) { + EmptyTableRef() : TableRef(TableReferenceType::EMPTY_FROM) { } public: diff --git a/src/duckdb/src/include/duckdb/parser/tableref/list.hpp b/src/duckdb/src/include/duckdb/parser/tableref/list.hpp index 5500ff220..c6606dfbc 100644 --- a/src/duckdb/src/include/duckdb/parser/tableref/list.hpp +++ b/src/duckdb/src/include/duckdb/parser/tableref/list.hpp @@ -3,5 +3,6 @@ #include "duckdb/parser/tableref/expressionlistref.hpp" #include "duckdb/parser/tableref/joinref.hpp" #include "duckdb/parser/tableref/pivotref.hpp" +#include "duckdb/parser/tableref/showref.hpp" #include "duckdb/parser/tableref/subqueryref.hpp" #include "duckdb/parser/tableref/table_function_ref.hpp" diff --git a/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp b/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp new file mode 100644 index 000000000..fdd98e21b --- /dev/null +++ b/src/duckdb/src/include/duckdb/parser/tableref/showref.hpp @@ -0,0 +1,47 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/parser/tableref/showref.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/parser/tableref.hpp" +#include "duckdb/parser/parsed_expression.hpp" +#include "duckdb/common/types.hpp" +#include "duckdb/common/vector.hpp" +#include "duckdb/parser/query_node.hpp" + +namespace duckdb { + +enum class ShowType : uint8_t { SUMMARY, DESCRIBE }; + +//! Represents a SHOW/DESCRIBE/SUMMARIZE statement +class ShowRef : public TableRef { +public: + static constexpr const TableReferenceType TYPE = TableReferenceType::SHOW_REF; + +public: + ShowRef(); + + //! The table name (if any) + string table_name; + //! The QueryNode of select query (if any) + unique_ptr query; + //! Whether or not we are requesting a summary or a describe + ShowType show_type; + +public: + string ToString() const override; + bool Equals(const TableRef &other_p) const override; + + unique_ptr Copy() override; + + //! Deserializes a blob back into a ExpressionListRef + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &source); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/parser/tokens.hpp b/src/duckdb/src/include/duckdb/parser/tokens.hpp index fa6402d7a..e9d3bb56c 100644 --- a/src/duckdb/src/include/duckdb/parser/tokens.hpp +++ b/src/duckdb/src/include/duckdb/parser/tokens.hpp @@ -31,7 +31,6 @@ class UpdateStatement; class PrepareStatement; class ExecuteStatement; class PragmaStatement; -class ShowStatement; class ExplainStatement; class ExportStatement; class VacuumStatement; @@ -98,6 +97,7 @@ class TableFunctionRef; class EmptyTableRef; class ExpressionListRef; class PivotRef; +class ShowRef; //===--------------------------------------------------------------------===// // Other diff --git a/src/duckdb/src/include/duckdb/parser/transformer.hpp b/src/duckdb/src/include/duckdb/parser/transformer.hpp index 710275197..28cdf40bf 100644 --- a/src/duckdb/src/include/duckdb/parser/transformer.hpp +++ b/src/duckdb/src/include/duckdb/parser/transformer.hpp @@ -16,6 +16,7 @@ #include "duckdb/common/unordered_map.hpp" #include "duckdb/parser/group_by_node.hpp" #include "duckdb/parser/parsed_data/create_info.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" #include "duckdb/parser/qualified_name.hpp" #include "duckdb/parser/query_node.hpp" #include "duckdb/parser/query_node/cte_node.hpp" @@ -74,7 +75,7 @@ class Transformer { //! Last parameter type PreparedParamType last_param_type = PreparedParamType::INVALID; //! Holds window expressions defined by name. We need those when transforming the expressions referring to them. - unordered_map window_clauses; + case_insensitive_map_t window_clauses; //! The set of pivot entries to create vector> pivot_entries; //! Sets of stored CTEs, if any @@ -132,6 +133,8 @@ class Transformer { unique_ptr TransformCreateFunction(duckdb_libpgquery::PGCreateFunctionStmt &stmt); //! Transform a Postgres duckdb_libpgquery::T_PGCreateTypeStmt node into CreateStatement unique_ptr TransformCreateType(duckdb_libpgquery::PGCreateTypeStmt &stmt); + //! Transform a Postgres duckdb_libpgquery::T_PGCreateTypeStmt node into CreateStatement + unique_ptr TransformCommentOn(duckdb_libpgquery::PGCommentOnStmt &stmt); //! Transform a Postgres duckdb_libpgquery::T_PGAlterSeqStmt node into CreateStatement unique_ptr TransformAlterSequence(duckdb_libpgquery::PGAlterSeqStmt &stmt); //! Transform a Postgres duckdb_libpgquery::T_PGDropStmt node into a Drop[Table,Schema]Statement @@ -148,6 +151,7 @@ class Transformer { //! Transform a Postgres duckdb_libpgquery::T_PGCopyStmt node into a CopyStatement unique_ptr TransformCopy(duckdb_libpgquery::PGCopyStmt &stmt); void TransformCopyOptions(CopyInfo &info, optional_ptr options); + void TransformCreateSecretOptions(CreateSecretInfo &info, optional_ptr options); //! Transform a Postgres duckdb_libpgquery::T_PGTransactionStmt node into a TransactionStatement unique_ptr TransformTransaction(duckdb_libpgquery::PGTransactionStmt &stmt); //! Transform a Postgres T_DeleteStatement node into a DeleteStatement @@ -162,12 +166,14 @@ class Transformer { unique_ptr TransformImport(duckdb_libpgquery::PGImportStmt &stmt); unique_ptr TransformExplain(duckdb_libpgquery::PGExplainStmt &stmt); unique_ptr TransformVacuum(duckdb_libpgquery::PGVacuumStmt &stmt); - unique_ptr TransformShow(duckdb_libpgquery::PGVariableShowStmt &stmt); - unique_ptr TransformShowSelect(duckdb_libpgquery::PGVariableShowSelectStmt &stmt); + unique_ptr TransformShow(duckdb_libpgquery::PGVariableShowStmt &stmt); + unique_ptr TransformShowSelect(duckdb_libpgquery::PGVariableShowSelectStmt &stmt); unique_ptr TransformAttach(duckdb_libpgquery::PGAttachStmt &stmt); unique_ptr TransformDetach(duckdb_libpgquery::PGDetachStmt &stmt); unique_ptr TransformUse(duckdb_libpgquery::PGUseStmt &stmt); unique_ptr TransformCopyDatabase(duckdb_libpgquery::PGCopyDatabaseStmt &stmt); + unique_ptr TransformSecret(duckdb_libpgquery::PGCreateSecretStmt &stmt); + unique_ptr TransformDropSecret(duckdb_libpgquery::PGDropSecretStmt &stmt); unique_ptr TransformPrepare(duckdb_libpgquery::PGPrepareStmt &stmt); unique_ptr TransformExecute(duckdb_libpgquery::PGExecuteStmt &stmt); @@ -346,6 +352,13 @@ class Transformer { Vector PGListToVector(optional_ptr column_list, idx_t &size); vector TransformConflictTarget(duckdb_libpgquery::PGList &list); + void ParseGenericOptionListEntry(case_insensitive_map_t> &result_options, string &name, + duckdb_libpgquery::PGNode *arg); + +public: + static void SetQueryLocation(ParsedExpression &expr, int query_location); + static void SetQueryLocation(TableRef &ref, int query_location); + private: //! Current stack depth idx_t stack_depth; diff --git a/src/duckdb/src/include/duckdb/planner/bind_context.hpp b/src/duckdb/src/include/duckdb/planner/bind_context.hpp index 64df2dbdd..d87a54a1c 100644 --- a/src/duckdb/src/include/duckdb/planner/bind_context.hpp +++ b/src/duckdb/src/include/duckdb/planner/bind_context.hpp @@ -11,6 +11,7 @@ #include "duckdb/catalog/catalog.hpp" #include "duckdb/common/case_insensitive_map.hpp" #include "duckdb/common/reference_map.hpp" +#include "duckdb/common/exception/binder_exception.hpp" #include "duckdb/parser/expression/columnref_expression.hpp" #include "duckdb/parser/parsed_expression.hpp" #include "duckdb/parser/qualified_name_set.hpp" @@ -147,7 +148,7 @@ class BindContext { //! Gets a binding of the specified name. Returns a nullptr and sets the out_error if the binding could not be //! found. - optional_ptr GetBinding(const string &name, string &out_error); + optional_ptr GetBinding(const string &name, ErrorData &out_error); private: void AddBinding(const string &alias, unique_ptr binding); diff --git a/src/duckdb/src/include/duckdb/planner/binder.hpp b/src/duckdb/src/include/duckdb/planner/binder.hpp index aa986a187..1d40a63a9 100644 --- a/src/duckdb/src/include/duckdb/planner/binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/binder.hpp @@ -12,6 +12,7 @@ #include "duckdb/common/enums/join_type.hpp" #include "duckdb/common/enums/statement_type.hpp" #include "duckdb/common/unordered_map.hpp" +#include "duckdb/common/exception/binder_exception.hpp" #include "duckdb/parser/column_definition.hpp" #include "duckdb/parser/query_node.hpp" #include "duckdb/parser/result_modifier.hpp" @@ -149,23 +150,6 @@ class Binder : public std::enable_shared_from_this { //! Add a correlated column to this binder (if it does not exist) void AddCorrelatedColumn(const CorrelatedColumnInfo &info); - string FormatError(ParsedExpression &expr_context, const string &message); - string FormatError(TableRef &ref_context, const string &message); - - string FormatErrorRecursive(idx_t query_location, const string &message, vector &values); - template - string FormatErrorRecursive(idx_t query_location, const string &msg, vector &values, T param, - ARGS... params) { - values.push_back(ExceptionFormatValue::CreateFormatValue(param)); - return FormatErrorRecursive(query_location, msg, values, params...); - } - - template - string FormatError(idx_t query_location, const string &msg, ARGS... params) { - vector values; - return FormatErrorRecursive(query_location, msg, values, params...); - } - unique_ptr BindUpdateSet(LogicalOperator &op, unique_ptr root, UpdateSetInfo &set_info, TableCatalogEntry &table, vector &columns); @@ -177,11 +161,11 @@ class Binder : public std::enable_shared_from_this { static void BindLogicalType(ClientContext &context, LogicalType &type, optional_ptr catalog = nullptr, const string &schema = INVALID_SCHEMA); - bool HasMatchingBinding(const string &table_name, const string &column_name, string &error_message); + bool HasMatchingBinding(const string &table_name, const string &column_name, ErrorData &error); bool HasMatchingBinding(const string &schema_name, const string &table_name, const string &column_name, - string &error_message); + ErrorData &error); bool HasMatchingBinding(const string &catalog_name, const string &schema_name, const string &table_name, - const string &column_name, string &error_message); + const string &column_name, ErrorData &error); void SetBindingMode(BindingMode mode); BindingMode GetBindingMode(); @@ -192,6 +176,7 @@ class Binder : public std::enable_shared_from_this { } void SetCanContainNulls(bool can_contain_nulls); + void SetAlwaysRequireRebind(); private: //! The parent binder (if any) @@ -253,7 +238,6 @@ class Binder : public std::enable_shared_from_this { BoundStatement Bind(ExplainStatement &stmt); BoundStatement Bind(VacuumStatement &stmt); BoundStatement Bind(RelationStatement &stmt); - BoundStatement Bind(ShowStatement &stmt); BoundStatement Bind(CallStatement &stmt); BoundStatement Bind(ExportStatement &stmt); BoundStatement Bind(ExtensionStatement &stmt); @@ -292,6 +276,7 @@ class Binder : public std::enable_shared_from_this { unique_ptr Bind(EmptyTableRef &ref); unique_ptr Bind(ExpressionListRef &ref); unique_ptr Bind(PivotRef &expr); + unique_ptr Bind(ShowRef &ref); unique_ptr BindPivot(PivotRef &expr, vector> all_columns); unique_ptr BindUnpivot(Binder &child_binder, PivotRef &expr, @@ -302,9 +287,9 @@ class Binder : public std::enable_shared_from_this { bool BindTableFunctionParameters(TableFunctionCatalogEntry &table_function, vector> &expressions, vector &arguments, vector ¶meters, named_parameter_map_t &named_parameters, - unique_ptr &subquery, string &error); + unique_ptr &subquery, ErrorData &error); bool BindTableInTableOutFunction(vector> &expressions, - unique_ptr &subquery, string &error); + unique_ptr &subquery, ErrorData &error); unique_ptr BindTableFunction(TableFunction &function, vector parameters); unique_ptr BindTableFunctionInternal(TableFunction &table_function, const string &function_name, vector parameters, @@ -328,7 +313,6 @@ class Binder : public std::enable_shared_from_this { void BindModifierTypes(BoundQueryNode &result, const vector &sql_types, idx_t projection_index, const vector &expansion_count = {}); - BoundStatement BindSummarize(ShowStatement &stmt); unique_ptr BindLimit(OrderBinder &order_binder, LimitModifier &limit_mod); unique_ptr BindLimitPercent(OrderBinder &order_binder, LimitPercentModifier &limit_mod); unique_ptr BindOrderExpression(OrderBinder &order_binder, unique_ptr expr); @@ -374,6 +358,10 @@ class Binder : public std::enable_shared_from_this { unique_ptr BindCopyDatabaseData(CopyDatabaseStatement &stmt, Catalog &from_database, Catalog &to_database); + unique_ptr BindShowQuery(ShowRef &ref); + unique_ptr BindShowTable(ShowRef &ref); + unique_ptr BindSummarize(ShowRef &ref); + public: // This should really be a private constructor, but make_shared does not allow it... // If you are thinking about calling this, you should probably call Binder::CreateBinder diff --git a/src/duckdb/src/include/duckdb/planner/bound_result_modifier.hpp b/src/duckdb/src/include/duckdb/planner/bound_result_modifier.hpp index a44922d73..31d3d612b 100644 --- a/src/duckdb/src/include/duckdb/planner/bound_result_modifier.hpp +++ b/src/duckdb/src/include/duckdb/planner/bound_result_modifier.hpp @@ -98,6 +98,10 @@ class BoundOrderModifier : public BoundResultModifier { void Serialize(Serializer &serializer) const; static unique_ptr Deserialize(Deserializer &deserializer); + + //! Remove unneeded/duplicate order elements. + //! Returns true of orders is not empty. + bool Simplify(const vector> &groups); }; enum class DistinctType : uint8_t { DISTINCT = 0, DISTINCT_ON = 1 }; diff --git a/src/duckdb/src/include/duckdb/planner/expression.hpp b/src/duckdb/src/include/duckdb/planner/expression.hpp index 208e9b5cb..3cfa825d8 100644 --- a/src/duckdb/src/include/duckdb/planner/expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression.hpp @@ -32,7 +32,8 @@ class Expression : public BaseExpression { bool HasSubquery() const override; bool IsScalar() const override; bool HasParameter() const override; - virtual bool HasSideEffects() const; + virtual bool IsVolatile() const; + virtual bool IsConsistent() const; virtual bool PropagatesNullValues() const; virtual bool IsFoldable() const; diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_comparison_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_comparison_expression.hpp index 9e34e3cf3..25ac664ce 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_comparison_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_comparison_expression.hpp @@ -33,6 +33,9 @@ class BoundComparisonExpression : public Expression { static unique_ptr Deserialize(Deserializer &deserializer); public: - static LogicalType BindComparison(LogicalType left_type, LogicalType right_type); + static LogicalType BindComparison(ClientContext &context, const LogicalType &left_type, + const LogicalType &right_type, ExpressionType comparison_type); + static bool TryBindComparison(ClientContext &context, const LogicalType &left_type, const LogicalType &right_type, + LogicalType &result_type, ExpressionType comparison_type); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/planner/expression/bound_function_expression.hpp b/src/duckdb/src/include/duckdb/planner/expression/bound_function_expression.hpp index 27201cb5c..34a87a4bd 100644 --- a/src/duckdb/src/include/duckdb/planner/expression/bound_function_expression.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression/bound_function_expression.hpp @@ -34,7 +34,8 @@ class BoundFunctionExpression : public Expression { bool is_operator; public: - bool HasSideEffects() const override; + bool IsVolatile() const override; + bool IsConsistent() const override; bool IsFoldable() const override; string ToString() const override; bool PropagatesNullValues() const override; 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 ff3e07c11..670c2b5c5 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 @@ -40,6 +40,8 @@ class BoundWindowExpression : public Expression { unique_ptr filter_expr; //! True to ignore NULL values bool ignore_nulls; + //! Whether or not the aggregate function is distinct, only used for aggregates + bool distinct; //! The window boundaries WindowBoundary start = WindowBoundary::INVALID; WindowBoundary end = WindowBoundary::INVALID; diff --git a/src/duckdb/src/include/duckdb/planner/expression_binder.hpp b/src/duckdb/src/include/duckdb/planner/expression_binder.hpp index 1c567181b..b74e0d289 100644 --- a/src/duckdb/src/include/duckdb/planner/expression_binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression_binder.hpp @@ -10,6 +10,8 @@ #include "duckdb/common/exception.hpp" #include "duckdb/common/stack_checker.hpp" +#include "duckdb/common/exception/binder_exception.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/unordered_map.hpp" #include "duckdb/parser/expression/bound_expression.hpp" #include "duckdb/parser/expression/lambdaref_expression.hpp" @@ -35,23 +37,30 @@ struct DummyBinding; struct BoundColumnReferenceInfo { string name; - idx_t query_location; + optional_idx query_location; }; struct BindResult { BindResult() { } - explicit BindResult(string error) : error(error) { + explicit BindResult(const Exception &ex) : error(ex) { + } + explicit BindResult(const string &error_msg) : error(ExceptionType::BINDER, error_msg) { + } + explicit BindResult(ErrorData error) : error(std::move(error)) { } explicit BindResult(unique_ptr expr) : expression(std::move(expr)) { } - bool HasError() { - return !error.empty(); + bool HasError() const { + return error.HasError(); + } + void SetError(const string &error_message) { + error = ErrorData(ExceptionType::BINDER, error_message); } unique_ptr expression; - string error; + ErrorData error; }; class ExpressionBinder { @@ -80,26 +89,37 @@ class ExpressionBinder { return bound_columns; } - string Bind(unique_ptr &expr, idx_t depth, bool root_expression = false); + ErrorData Bind(unique_ptr &expr, idx_t depth, bool root_expression = false); - unique_ptr CreateStructExtract(unique_ptr base, string field_name); - unique_ptr CreateStructPack(ColumnRefExpression &colref); - BindResult BindQualifiedColumnName(ColumnRefExpression &colref, const string &table_name); + //! Returns the STRUCT_EXTRACT operator expression + unique_ptr CreateStructExtract(unique_ptr base, const string &field_name); + //! Returns a STRUCT_PACK function expression + unique_ptr CreateStructPack(ColumnRefExpression &col_ref); - unique_ptr QualifyColumnName(const string &column_name, string &error_message); - unique_ptr QualifyColumnName(ColumnRefExpression &colref, string &error_message); + BindResult BindQualifiedColumnName(ColumnRefExpression &colref, const string &table_name); - // Bind table names to ColumnRefExpressions - void QualifyColumnNames(unique_ptr &expr); + //! Returns a qualified column reference from a column name + unique_ptr QualifyColumnName(const string &column_name, ErrorData &error); + //! Returns a qualified column reference from a column reference with column_names.size() > 2 + unique_ptr QualifyColumnNameWithManyDots(ColumnRefExpression &col_ref, ErrorData &error); + //! Returns a qualified column reference from a column reference + unique_ptr QualifyColumnName(ColumnRefExpression &col_ref, ErrorData &error); + //! Enables special-handling of lambda parameters by tracking them in the lambda_params vector + void QualifyColumnNamesInLambda(FunctionExpression &function, vector> &lambda_params); + //! Recursively qualifies the column references in the (children) of the expression. Passes on the + //! within_function_expression state from outer expressions, or sets it + void QualifyColumnNames(unique_ptr &expr, vector> &lambda_params, + const bool within_function_expression = false); + //! Entry point for qualifying the column references of the expression static void QualifyColumnNames(Binder &binder, unique_ptr &expr); static bool PushCollation(ClientContext &context, unique_ptr &source, const LogicalType &sql_type, bool equality_only = false); static void TestCollation(ClientContext &context, const string &collation); - BindResult BindCorrelatedColumns(unique_ptr &expr, string error_message); + BindResult BindCorrelatedColumns(unique_ptr &expr, ErrorData error_message); - void BindChild(unique_ptr &expr, idx_t depth, string &error); + void BindChild(unique_ptr &expr, idx_t depth, ErrorData &error); static void ExtractCorrelatedExpressions(Binder &binder, Expression &expr); static bool ContainsNullType(const LogicalType &type); @@ -109,12 +129,17 @@ class ExpressionBinder { virtual bool QualifyColumnAlias(const ColumnRefExpression &colref); - //! Bind the given expresion. Unlike Bind(), this does *not* mute the given ParsedExpression. + //! Bind the given expression. Unlike Bind(), this does *not* mute the given ParsedExpression. //! Exposed to be used from sub-binders that aren't subclasses of ExpressionBinder. virtual BindResult BindExpression(unique_ptr &expr_ptr, idx_t depth, bool root_expression = false); - void ReplaceMacroParametersRecursive(unique_ptr &expr); + //! Recursively replaces macro parameters with the provided input parameters + void ReplaceMacroParameters(unique_ptr &expr, vector> &lambda_params); + //! Enables special-handling of lambda parameters by tracking them in the lambda_params vector + void ReplaceMacroParametersInLambda(FunctionExpression &function, vector> &lambda_params); + + static LogicalType GetExpressionReturnType(const Expression &expr); private: //! Maximum stack depth @@ -153,6 +178,10 @@ class ExpressionBinder { static unique_ptr GetSQLValueFunction(const string &column_name); + LogicalType ResolveOperatorType(OperatorExpression &op, vector> &children); + LogicalType ResolveInType(OperatorExpression &op, vector> &children); + LogicalType ResolveNotType(OperatorExpression &op, vector> &children); + protected: virtual BindResult BindGroupingFunction(OperatorExpression &op, idx_t depth); virtual BindResult BindFunction(FunctionExpression &expr, ScalarFunctionCatalogEntry &function, idx_t depth); @@ -169,6 +198,13 @@ class ExpressionBinder { ClientContext &context; optional_ptr stored_binder; vector bound_columns; + + //! Returns true if the function name is an alias for the UNNEST function + static bool IsUnnestFunction(const string &function_name); + //! Returns true, if the function contains a lambda expression and is not the '->>' operator + static bool IsLambdaFunction(const FunctionExpression &function); + //! Returns the bind result of binding a lambda or JSON function + BindResult TryBindLambdaOrJson(FunctionExpression &function, idx_t depth, CatalogEntry &func); }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/planner/expression_binder/column_alias_binder.hpp b/src/duckdb/src/include/duckdb/planner/expression_binder/column_alias_binder.hpp index ce95ac73c..91476fe02 100644 --- a/src/duckdb/src/include/duckdb/planner/expression_binder/column_alias_binder.hpp +++ b/src/duckdb/src/include/duckdb/planner/expression_binder/column_alias_binder.hpp @@ -21,8 +21,8 @@ class ColumnAliasBinder { public: ColumnAliasBinder(BoundSelectNode &node, const case_insensitive_map_t &alias_map); - BindResult BindAlias(ExpressionBinder &enclosing_binder, ColumnRefExpression &expr, idx_t depth, - bool root_expression); + bool BindAlias(ExpressionBinder &enclosing_binder, ColumnRefExpression &expr, idx_t depth, bool root_expression, + BindResult &result); private: BoundSelectNode &node; diff --git a/src/duckdb/src/include/duckdb/planner/filter/struct_filter.hpp b/src/duckdb/src/include/duckdb/planner/filter/struct_filter.hpp new file mode 100644 index 000000000..b7ae1711e --- /dev/null +++ b/src/duckdb/src/include/duckdb/planner/filter/struct_filter.hpp @@ -0,0 +1,41 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/planner/filter/constant_filter.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/planner/table_filter.hpp" +#include "duckdb/common/types/value.hpp" +#include "duckdb/common/enums/expression_type.hpp" + +namespace duckdb { + +class StructFilter : public TableFilter { +public: + static constexpr const TableFilterType TYPE = TableFilterType::STRUCT_EXTRACT; + +public: + StructFilter(idx_t child_idx, string child_name, unique_ptr child_filter); + + //! The field index to filter on + idx_t child_idx; + + //! The field name to filter on + string child_name; + + //! The child filter + unique_ptr child_filter; + +public: + FilterPropagateResult CheckStatistics(BaseStatistics &stats) override; + string ToString(const string &column_name) override; + bool Equals(const TableFilter &other) const override; + void Serialize(Serializer &serializer) const override; + static unique_ptr Deserialize(Deserializer &deserializer); +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/planner/logical_tokens.hpp b/src/duckdb/src/include/duckdb/planner/logical_tokens.hpp index 3bb0f5232..7854bd393 100644 --- a/src/duckdb/src/include/duckdb/planner/logical_tokens.hpp +++ b/src/duckdb/src/include/duckdb/planner/logical_tokens.hpp @@ -22,6 +22,7 @@ class LogicalCreate; class LogicalCreateTable; class LogicalCreateIndex; class LogicalCreateTable; +class LogicalCreateSecret; class LogicalCrossProduct; class LogicalCTERef; class LogicalDelete; @@ -48,7 +49,6 @@ class LogicalRecursiveCTE; class LogicalMaterializedCTE; class LogicalSetOperation; class LogicalSample; -class LogicalShow; class LogicalSimple; class LogicalSet; class LogicalReset; diff --git a/src/duckdb/src/include/duckdb/planner/operator/list.hpp b/src/duckdb/src/include/duckdb/planner/operator/list.hpp index ee3125f7a..c0dfd0c8a 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/list.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/list.hpp @@ -7,6 +7,7 @@ #include "duckdb/planner/operator/logical_create.hpp" #include "duckdb/planner/operator/logical_create_index.hpp" #include "duckdb/planner/operator/logical_create_table.hpp" +#include "duckdb/planner/operator/logical_create_secret.hpp" #include "duckdb/planner/operator/logical_cross_product.hpp" #include "duckdb/planner/operator/logical_cteref.hpp" #include "duckdb/planner/operator/logical_delete.hpp" @@ -37,7 +38,6 @@ #include "duckdb/planner/operator/logical_sample.hpp" #include "duckdb/planner/operator/logical_set.hpp" #include "duckdb/planner/operator/logical_set_operation.hpp" -#include "duckdb/planner/operator/logical_show.hpp" #include "duckdb/planner/operator/logical_simple.hpp" #include "duckdb/planner/operator/logical_top_n.hpp" #include "duckdb/planner/operator/logical_unnest.hpp" diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_comparison_join.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_comparison_join.hpp index abcfafb0b..4cb62ca33 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_comparison_join.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_comparison_join.hpp @@ -31,6 +31,8 @@ class LogicalComparisonJoin : public LogicalJoin { vector mark_types; //! The set of columns that will be duplicate eliminated from the LHS and pushed into the RHS vector> duplicate_eliminated_columns; + //! If this is a DelimJoin, whether it has been flipped to de-duplicating the RHS instead + bool delim_flipped = false; public: string ParamsToString() const override; diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_copy_to_file.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_copy_to_file.hpp index 0e1a7789d..ea79b9407 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_copy_to_file.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_copy_to_file.hpp @@ -10,6 +10,7 @@ #include "duckdb/common/filename_pattern.hpp" #include "duckdb/common/local_file_system.hpp" +#include "duckdb/common/optional_idx.hpp" #include "duckdb/function/copy_function.hpp" #include "duckdb/planner/logical_operator.hpp" @@ -31,8 +32,10 @@ class LogicalCopyToFile : public LogicalOperator { std::string file_path; bool use_tmp_file; FilenamePattern filename_pattern; + string file_extension; bool overwrite_or_ignore; bool per_thread_output; + optional_idx file_size_bytes; bool partition_output; vector partition_columns; diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_create_secret.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_create_secret.hpp new file mode 100644 index 000000000..1885e48cd --- /dev/null +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_create_secret.hpp @@ -0,0 +1,43 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// planner/operator/logical_create_secret.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/parser/parsed_data/create_secret_info.hpp" +#include "duckdb/planner/logical_operator.hpp" + +namespace duckdb { + +//! LogicalCreateSecret represents a simple logical operator that only passes on the parse info +class LogicalCreateSecret : public LogicalOperator { +public: + static constexpr const LogicalOperatorType TYPE = LogicalOperatorType::LOGICAL_CREATE_SECRET; + +public: + LogicalCreateSecret(CreateSecretFunction function_p, CreateSecretInfo info_p) + : LogicalOperator(LogicalOperatorType::LOGICAL_CREATE_SECRET), info(std::move(info_p)) { + } + + CreateSecretInfo info; + +public: + idx_t EstimateCardinality(ClientContext &context) override { + return 1; + }; + + //! Skips the serialization check in VerifyPlan + bool SupportSerialization() const override { + return false; + } + +protected: + void ResolveTypes() override { + types.emplace_back(LogicalType::BOOLEAN); + } +}; +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_cteref.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_cteref.hpp index da943b01d..5fb2debff 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_cteref.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_cteref.hpp @@ -8,7 +8,6 @@ #pragma once -#include "duckdb/common/types/chunk_collection.hpp" #include "duckdb/planner/logical_operator.hpp" #include "duckdb/common/enums/cte_materialize.hpp" diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_pragma.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_pragma.hpp index 576d5dd17..87e08fac0 100644 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_pragma.hpp +++ b/src/duckdb/src/include/duckdb/planner/operator/logical_pragma.hpp @@ -13,7 +13,7 @@ namespace duckdb { -//! LogicalSimple represents a simple logical operator that only passes on the parse info +//! LogicalPragma represents a simple logical operator that only passes on the parse info class LogicalPragma : public LogicalOperator { public: static constexpr const LogicalOperatorType TYPE = LogicalOperatorType::LOGICAL_PRAGMA; diff --git a/src/duckdb/src/include/duckdb/planner/operator/logical_show.hpp b/src/duckdb/src/include/duckdb/planner/operator/logical_show.hpp deleted file mode 100644 index f3b5ee103..000000000 --- a/src/duckdb/src/include/duckdb/planner/operator/logical_show.hpp +++ /dev/null @@ -1,42 +0,0 @@ -//===----------------------------------------------------------------------===// -// DuckDB -// -// duckdb/planner/operator/logical_show.hpp -// -// -//===----------------------------------------------------------------------===// - -#pragma once - -#include "duckdb/planner/logical_operator.hpp" - -namespace duckdb { - -class LogicalShow : public LogicalOperator { - LogicalShow() : LogicalOperator(LogicalOperatorType::LOGICAL_SHOW) {}; - -public: - static constexpr const LogicalOperatorType TYPE = LogicalOperatorType::LOGICAL_SHOW; - -public: - explicit LogicalShow(unique_ptr plan) : LogicalOperator(LogicalOperatorType::LOGICAL_SHOW) { - children.push_back(std::move(plan)); - } - - vector types_select; - vector aliases; - -public: - void Serialize(Serializer &serializer) const override; - static unique_ptr Deserialize(Deserializer &deserializer); - -protected: - void ResolveTypes() override { - types = {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR, - LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}; - } - vector GetColumnBindings() override { - return GenerateColumnBindings(0, types.size()); - } -}; -} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/planner/table_binding.hpp b/src/duckdb/src/include/duckdb/planner/table_binding.hpp index c282deca5..aa288a12b 100644 --- a/src/duckdb/src/include/duckdb/planner/table_binding.hpp +++ b/src/duckdb/src/include/duckdb/planner/table_binding.hpp @@ -52,7 +52,7 @@ struct Binding { bool TryGetBindingIndex(const string &column_name, column_t &column_index); column_t GetBindingIndex(const string &column_name); bool HasMatchingBinding(const string &column_name); - virtual string ColumnNotFoundError(const string &column_name) const; + virtual ErrorData ColumnNotFoundError(const string &column_name) const; virtual BindResult Bind(ColumnRefExpression &colref, idx_t depth); virtual optional_ptr GetStandardEntry(); @@ -107,7 +107,7 @@ struct TableBinding : public Binding { unique_ptr ExpandGeneratedColumn(const string &column_name); BindResult Bind(ColumnRefExpression &colref, idx_t depth) override; optional_ptr GetStandardEntry() override; - string ColumnNotFoundError(const string &column_name) const override; + ErrorData ColumnNotFoundError(const string &column_name) const override; // These are columns that are present in the name_map, appearing in the order that they're bound const vector &GetBoundColumnIds() const; diff --git a/src/duckdb/src/include/duckdb/planner/table_filter.hpp b/src/duckdb/src/include/duckdb/planner/table_filter.hpp index 368b13eda..2494b57bc 100644 --- a/src/duckdb/src/include/duckdb/planner/table_filter.hpp +++ b/src/duckdb/src/include/duckdb/planner/table_filter.hpp @@ -21,7 +21,8 @@ enum class TableFilterType : uint8_t { IS_NULL = 1, IS_NOT_NULL = 2, CONJUNCTION_OR = 3, - CONJUNCTION_AND = 4 + CONJUNCTION_AND = 4, + STRUCT_EXTRACT = 5 }; //! TableFilter represents a filter pushed down into the table scan. diff --git a/src/duckdb/src/include/duckdb/planner/tableref/bound_dummytableref.hpp b/src/duckdb/src/include/duckdb/planner/tableref/bound_dummytableref.hpp index debeb6e27..3a68f5166 100644 --- a/src/duckdb/src/include/duckdb/planner/tableref/bound_dummytableref.hpp +++ b/src/duckdb/src/include/duckdb/planner/tableref/bound_dummytableref.hpp @@ -15,10 +15,11 @@ namespace duckdb { //! Represents a cross product class BoundEmptyTableRef : public BoundTableRef { public: - static constexpr const TableReferenceType TYPE = TableReferenceType::EMPTY; + static constexpr const TableReferenceType TYPE = TableReferenceType::EMPTY_FROM; public: - explicit BoundEmptyTableRef(idx_t bind_index) : BoundTableRef(TableReferenceType::EMPTY), bind_index(bind_index) { + explicit BoundEmptyTableRef(idx_t bind_index) + : BoundTableRef(TableReferenceType::EMPTY_FROM), bind_index(bind_index) { } idx_t bind_index; }; diff --git a/src/duckdb/src/include/duckdb/storage/buffer/buffer_pool.hpp b/src/duckdb/src/include/duckdb/storage/buffer/buffer_pool.hpp index ea9c732fd..b2dfa6731 100644 --- a/src/duckdb/src/include/duckdb/storage/buffer/buffer_pool.hpp +++ b/src/duckdb/src/include/duckdb/storage/buffer/buffer_pool.hpp @@ -1,11 +1,20 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/buffer/buffer_pool.hpp +// +// +//===----------------------------------------------------------------------===// + #pragma once -#include "duckdb/common/mutex.hpp" #include "duckdb/common/file_buffer.hpp" +#include "duckdb/common/mutex.hpp" #include "duckdb/storage/buffer/block_handle.hpp" namespace duckdb { +class TemporaryMemoryManager; struct EvictionQueue; struct BufferEvictionNode { @@ -46,6 +55,10 @@ class BufferPool { idx_t GetMaxMemory() const; + virtual idx_t GetQueryMaxMemory() const; + + TemporaryMemoryManager &GetTemporaryMemoryManager(); + protected: //! Evict blocks until the currently used memory + extra_memory fit, returns false if this was not possible //! (i.e. not enough blocks could be evicted) @@ -75,6 +88,8 @@ class BufferPool { unique_ptr queue; //! Total number of insertions into the eviction queue. This guides the schedule for calling PurgeQueue. atomic queue_insertions; + //! Memory manager for concurrently used temporary memory, e.g., for physical operators + unique_ptr temporary_memory_manager; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/buffer_manager.hpp b/src/duckdb/src/include/duckdb/storage/buffer_manager.hpp index 614d4615b..6c021c2ec 100644 --- a/src/duckdb/src/include/duckdb/storage/buffer_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/buffer_manager.hpp @@ -19,6 +19,7 @@ namespace duckdb { class Allocator; class BufferPool; +class TemporaryMemoryManager; class BufferManager { friend class BufferHandle; @@ -59,7 +60,9 @@ class BufferManager { virtual unique_ptr ConstructManagedBuffer(idx_t size, unique_ptr &&source, FileBufferType type = FileBufferType::MANAGED_BUFFER); //! Get the underlying buffer pool responsible for managing the buffers - virtual BufferPool &GetBufferPool(); + virtual BufferPool &GetBufferPool() const; + //! Get the manager that assigns reservations for temporary memory, e.g., for query intermediates + virtual TemporaryMemoryManager &GetTemporaryMemoryManager(); // Static methods DUCKDB_API static BufferManager &GetBufferManager(DatabaseInstance &db); @@ -69,6 +72,8 @@ class BufferManager { static idx_t GetAllocSize(idx_t block_size) { return AlignValue(block_size + Storage::BLOCK_HEADER_SIZE); } + //! Returns the maximum available memory for a given query + idx_t GetQueryMaxMemory() const; protected: virtual void PurgeQueue() = 0; diff --git a/src/duckdb/src/include/duckdb/storage/checkpoint/string_checkpoint_state.hpp b/src/duckdb/src/include/duckdb/storage/checkpoint/string_checkpoint_state.hpp index f9de61632..fd3568ac9 100644 --- a/src/duckdb/src/include/duckdb/storage/checkpoint/string_checkpoint_state.hpp +++ b/src/duckdb/src/include/duckdb/storage/checkpoint/string_checkpoint_state.hpp @@ -39,7 +39,7 @@ struct string_location_t { string_location_t() { } bool IsValid() { - return offset < Storage::BLOCK_SIZE && (block_id == INVALID_BLOCK || block_id >= MAXIMUM_BLOCK); + return offset < int32_t(Storage::BLOCK_SIZE) && (block_id == INVALID_BLOCK || block_id >= MAXIMUM_BLOCK); } block_id_t block_id; int32_t offset; diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/algorithm/alp.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/algorithm/alp.hpp new file mode 100644 index 000000000..49eb80815 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/algorithm/alp.hpp @@ -0,0 +1,408 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/algorithm/alp.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/bitpacking.hpp" +#include "duckdb/common/common.hpp" +#include "duckdb/common/unordered_map.hpp" +#include "duckdb/common/pair.hpp" +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/hash.hpp" +#include "duckdb/storage/compression/alp/alp_constants.hpp" +#include "duckdb/storage/compression/alp/alp_utils.hpp" + +#include + +namespace duckdb { + +namespace alp { + +struct AlpEncodingIndices { + uint8_t exponent; + uint8_t factor; + + AlpEncodingIndices(uint8_t exponent, uint8_t factor) : exponent(exponent), factor(factor) { + } + + AlpEncodingIndices() : exponent(0), factor(0) { + } +}; + +struct AlpEncodingIndicesEquality { + bool operator()(const AlpEncodingIndices &a, const AlpEncodingIndices &b) const { + return a.exponent == b.exponent && a.factor == b.factor; + } +}; + +struct AlpEncodingIndicesHash { + hash_t operator()(const AlpEncodingIndices &encoding_indices) const { + hash_t h1 = Hash(encoding_indices.exponent); + hash_t h2 = Hash(encoding_indices.factor); + return CombineHash(h1, h2); + } +}; + +struct AlpCombination { + AlpEncodingIndices encoding_indices; + uint64_t n_appearances; + uint64_t estimated_compression_size; + + AlpCombination(AlpEncodingIndices encoding_indices, uint64_t n_appearances, uint64_t estimated_compression_size) + : encoding_indices(encoding_indices), n_appearances(n_appearances), + estimated_compression_size(estimated_compression_size) { + } +}; + +template +class AlpCompressionState { +public: + AlpCompressionState() : vector_encoding_indices(0, 0), exceptions_count(0), bit_width(0) { + } + + void Reset() { + vector_encoding_indices = {0, 0}; + exceptions_count = 0; + bit_width = 0; + } + + void ResetCombinations() { + best_k_combinations.clear(); + } + +public: + AlpEncodingIndices vector_encoding_indices; + uint16_t exceptions_count; + uint16_t bit_width; + uint64_t bp_size; + uint64_t frame_of_reference; + int64_t encoded_integers[AlpConstants::ALP_VECTOR_SIZE]; + T exceptions[AlpConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions_positions[AlpConstants::ALP_VECTOR_SIZE]; + vector best_k_combinations; + uint8_t values_encoded[AlpConstants::ALP_VECTOR_SIZE * 8]; +}; + +template +struct AlpCompression { + using State = AlpCompressionState; + static constexpr uint8_t EXACT_TYPE_BITSIZE = sizeof(T) * 8; + + /* + * Check for special values which are impossible for ALP to encode + * because they cannot be cast to int64 without an undefined behaviour + */ + static bool IsImpossibleToEncode(T n) { + return !Value::IsFinite(n) || Value::IsNan(n) || n > AlpConstants::ENCODING_UPPER_LIMIT || + n < AlpConstants::ENCODING_LOWER_LIMIT || (n == 0.0 && std::signbit(n)); //! Verification for -0.0 + } + + /* + * Conversion from a Floating-Point number to Int64 without rounding + */ + static int64_t NumberToInt64(T n) { + if (IsImpossibleToEncode(n)) { + return AlpConstants::ENCODING_UPPER_LIMIT; + } + n = n + AlpTypedConstants::MAGIC_NUMBER - AlpTypedConstants::MAGIC_NUMBER; + return static_cast(n); + } + + /* + * Encoding a single value with ALP + */ + static int64_t EncodeValue(T value, AlpEncodingIndices encoding_indices) { + T tmp_encoded_value = value * AlpTypedConstants::EXP_ARR[encoding_indices.exponent] * + AlpTypedConstants::FRAC_ARR[encoding_indices.factor]; + int64_t encoded_value = NumberToInt64(tmp_encoded_value); + return encoded_value; + } + + /* + * Decoding a single value with ALP + */ + static T DecodeValue(int64_t encoded_value, AlpEncodingIndices encoding_indices) { + //! The cast to T is needed to prevent a signed integer overflow + T decoded_value = static_cast(encoded_value) * AlpConstants::FACT_ARR[encoding_indices.factor] * + AlpTypedConstants::FRAC_ARR[encoding_indices.exponent]; + return decoded_value; + } + + /* + * Return TRUE if c1 is a better combination than c2 + * First criteria is number of times it appears as best combination + * Second criteria is the estimated compression size + * Third criteria is bigger exponent + * Fourth criteria is bigger factor + */ + static bool CompareALPCombinations(const AlpCombination &c1, const AlpCombination &c2) { + return (c1.n_appearances > c2.n_appearances) || + (c1.n_appearances == c2.n_appearances && + (c1.estimated_compression_size < c2.estimated_compression_size)) || + ((c1.n_appearances == c2.n_appearances && + c1.estimated_compression_size == c2.estimated_compression_size) && + (c2.encoding_indices.exponent < c1.encoding_indices.exponent)) || + ((c1.n_appearances == c2.n_appearances && + c1.estimated_compression_size == c2.estimated_compression_size && + c2.encoding_indices.exponent == c1.encoding_indices.exponent) && + (c2.encoding_indices.factor < c1.encoding_indices.factor)); + } + + /* + * Dry compress a vector (ideally a sample) to estimate ALP compression size given a exponent and factor + */ + template + static uint64_t DryCompressToEstimateSize(const vector &input_vector, AlpEncodingIndices encoding_indices) { + idx_t n_values = input_vector.size(); + idx_t exceptions_count = 0; + idx_t non_exceptions_count = 0; + uint32_t estimated_bits_per_value = 0; + uint64_t estimated_compression_size = 0; + int64_t max_encoded_value = NumericLimits::Minimum(); + int64_t min_encoded_value = NumericLimits::Maximum(); + + for (const T &value : input_vector) { + int64_t encoded_value = EncodeValue(value, encoding_indices); + T decoded_value = DecodeValue(encoded_value, encoding_indices); + if (decoded_value == value) { + non_exceptions_count++; + max_encoded_value = MaxValue(encoded_value, max_encoded_value); + min_encoded_value = MinValue(encoded_value, min_encoded_value); + continue; + } + exceptions_count++; + } + + // We penalize combinations which yields to almost all exceptions + if (PENALIZE_EXCEPTIONS && non_exceptions_count < 2) { + return NumericLimits::Maximum(); + } + + // Evaluate factor/exponent compression size (we optimize for FOR) + uint64_t delta = (static_cast(max_encoded_value) - static_cast(min_encoded_value)); + estimated_bits_per_value = std::ceil(std::log2(delta + 1)); + estimated_compression_size += n_values * estimated_bits_per_value; + estimated_compression_size += + exceptions_count * (EXACT_TYPE_BITSIZE + (AlpConstants::EXCEPTION_POSITION_SIZE * 8)); + return estimated_compression_size; + } + + /* + * Find the best combinations of factor-exponent from each vector sampled from a rowgroup + * This function is called once per segment + * This operates over ALP first level samples + */ + static void FindTopKCombinations(const vector> &vectors_sampled, State &state) { + state.ResetCombinations(); + + unordered_map + best_k_combinations_hash; + // For each vector sampled + for (auto &sampled_vector : vectors_sampled) { + idx_t n_samples = sampled_vector.size(); + AlpEncodingIndices best_encoding_indices = {AlpTypedConstants::MAX_EXPONENT, + AlpTypedConstants::MAX_EXPONENT}; + + //! We start our optimization with the worst possible total bits obtained from compression + idx_t best_total_bits = (n_samples * (EXACT_TYPE_BITSIZE + AlpConstants::EXCEPTION_POSITION_SIZE * 8)) + + (n_samples * EXACT_TYPE_BITSIZE); + + // N of appearances is irrelevant at this phase; we search for the best compression for the vector + AlpCombination best_combination = {best_encoding_indices, 0, best_total_bits}; + //! We try all combinations in search for the one which minimize the compression size + for (int8_t exp_idx = AlpTypedConstants::MAX_EXPONENT; exp_idx >= 0; exp_idx--) { + for (int8_t factor_idx = exp_idx; factor_idx >= 0; factor_idx--) { + AlpEncodingIndices current_encoding_indices = {(uint8_t)exp_idx, (uint8_t)factor_idx}; + uint64_t estimated_compression_size = + DryCompressToEstimateSize(sampled_vector, current_encoding_indices); + AlpCombination current_combination = {current_encoding_indices, 0, estimated_compression_size}; + if (CompareALPCombinations(current_combination, best_combination)) { + best_combination = current_combination; + } + } + } + best_k_combinations_hash[best_combination.encoding_indices]++; + } + + // Convert our hash to a Combination vector to be able to sort + // Note that this vector is always small (< 10 combinations) + vector best_k_combinations; + for (auto const &combination : best_k_combinations_hash) { + best_k_combinations.emplace_back( + combination.first, // Encoding Indices + combination.second, // N of times it appeared (hash value) + 0 // Compression size is irrelevant at this phase since we compare combinations from different vectors + ); + } + sort(best_k_combinations.begin(), best_k_combinations.end(), CompareALPCombinations); + + // Save k' best combinations + for (idx_t i = 0; i < MinValue(AlpConstants::MAX_COMBINATIONS, (uint8_t)best_k_combinations.size()); i++) { + state.best_k_combinations.push_back(best_k_combinations[i]); + } + } + + /* + * Find the best combination of factor-exponent for a vector from within the best k combinations + * This is ALP second level sampling + */ + static void FindBestFactorAndExponent(const T *input_vector, idx_t n_values, State &state) { + //! We sample equidistant values within a vector; to do this we skip a fixed number of values + vector vector_sample; + uint32_t idx_increments = MaxValue(1, (int32_t)std::ceil((double)n_values / AlpConstants::SAMPLES_PER_VECTOR)); + for (idx_t i = 0; i < n_values; i += idx_increments) { + vector_sample.push_back(input_vector[i]); + } + + AlpEncodingIndices best_encoding_indices = {0, 0}; + uint64_t best_total_bits = NumericLimits::Maximum(); + idx_t worse_total_bits_counter = 0; + + //! We try each K combination in search for the one which minimize the compression size in the vector + for (auto &combination : state.best_k_combinations) { + uint64_t estimated_compression_size = + DryCompressToEstimateSize(vector_sample, combination.encoding_indices); + + // If current compression size is worse (higher) or equal than the current best combination + if (estimated_compression_size >= best_total_bits) { + worse_total_bits_counter += 1; + // Early exit strategy + if (worse_total_bits_counter == AlpConstants::SAMPLING_EARLY_EXIT_THRESHOLD) { + break; + } + continue; + } + // Otherwise we replace the best and continue trying with the next combination + best_total_bits = estimated_compression_size; + best_encoding_indices = combination.encoding_indices; + worse_total_bits_counter = 0; + } + state.vector_encoding_indices = best_encoding_indices; + } + + /* + * ALP Compress + */ + static void Compress(const T *input_vector, idx_t n_values, const uint16_t *vector_null_positions, + idx_t nulls_count, State &state) { + if (state.best_k_combinations.size() > 1) { + FindBestFactorAndExponent(input_vector, n_values, state); + } else { + state.vector_encoding_indices = state.best_k_combinations[0].encoding_indices; + } + + // Encoding Floating-Point to Int64 + //! We encode all the values regardless of their correctness to recover the original floating-point + uint16_t exceptions_idx = 0; + for (idx_t i = 0; i < n_values; i++) { + T actual_value = input_vector[i]; + int64_t encoded_value = EncodeValue(actual_value, state.vector_encoding_indices); + T decoded_value = DecodeValue(encoded_value, state.vector_encoding_indices); + state.encoded_integers[i] = encoded_value; + //! We detect exceptions using a predicated comparison + auto is_exception = (decoded_value != actual_value); + state.exceptions_positions[exceptions_idx] = i; + exceptions_idx += is_exception; + } + + // Finding first non exception value + int64_t a_non_exception_value = 0; + for (idx_t i = 0; i < n_values; i++) { + if (i != state.exceptions_positions[i]) { + a_non_exception_value = state.encoded_integers[i]; + break; + } + } + // Replacing that first non exception value on the vector exceptions + for (idx_t i = 0; i < exceptions_idx; i++) { + idx_t exception_pos = state.exceptions_positions[i]; + T actual_value = input_vector[exception_pos]; + state.encoded_integers[exception_pos] = a_non_exception_value; + state.exceptions[i] = actual_value; + } + state.exceptions_count = exceptions_idx; + + // Replacing nulls with that first non exception value + for (idx_t i = 0; i < nulls_count; i++) { + uint16_t null_value_pos = vector_null_positions[i]; + state.encoded_integers[null_value_pos] = a_non_exception_value; + } + + // Analyze FFOR + auto min_value = NumericLimits::Maximum(); + auto max_value = NumericLimits::Minimum(); + for (idx_t i = 0; i < n_values; i++) { + max_value = MaxValue(max_value, state.encoded_integers[i]); + min_value = MinValue(min_value, state.encoded_integers[i]); + } + uint64_t min_max_diff = (static_cast(max_value) - static_cast(min_value)); + + auto *u_encoded_integers = reinterpret_cast(state.encoded_integers); + auto const u_min_value = static_cast(min_value); + + // Subtract FOR + if (!EMPTY) { //! We only execute the FOR if we are writing the data + for (idx_t i = 0; i < n_values; i++) { + u_encoded_integers[i] -= u_min_value; + } + } + + auto bit_width = BitpackingPrimitives::MinimumBitWidth(min_max_diff); + auto bp_size = BitpackingPrimitives::GetRequiredSize(n_values, bit_width); + if (!EMPTY && bit_width > 0) { //! We only execute the BP if we are writing the data + BitpackingPrimitives::PackBuffer(state.values_encoded, u_encoded_integers, n_values, + bit_width); + } + state.bit_width = bit_width; // in bits + state.bp_size = bp_size; // in bytes + state.frame_of_reference = min_value; + } + + /* + * Overload without specifying nulls + */ + static void Compress(const T *input_vector, idx_t n_values, State &state) { + Compress(input_vector, n_values, nullptr, 0, state); + } +}; + +template +struct AlpDecompression { + static void Decompress(uint8_t *for_encoded, T *output, idx_t count, uint8_t vector_factor, uint8_t vector_exponent, + uint16_t exceptions_count, T *exceptions, const uint16_t *exceptions_positions, + uint64_t frame_of_reference, uint8_t bit_width) { + AlpEncodingIndices encoding_indices = {vector_exponent, vector_factor}; + + // Bit Unpacking + uint8_t for_decoded[AlpConstants::ALP_VECTOR_SIZE * 8] = {0}; + if (bit_width > 0) { + BitpackingPrimitives::UnPackBuffer(for_decoded, for_encoded, count, bit_width); + } + auto *encoded_integers = reinterpret_cast(data_ptr_cast(for_decoded)); + + // unFOR + for (idx_t i = 0; i < count; i++) { + encoded_integers[i] += frame_of_reference; + } + + // Decoding + for (idx_t i = 0; i < count; i++) { + auto encoded_integer = static_cast(encoded_integers[i]); + output[i] = alp::AlpCompression::DecodeValue(encoded_integer, encoding_indices); + } + + // Exceptions Patching + for (idx_t i = 0; i < exceptions_count; i++) { + output[exceptions_positions[i]] = static_cast(exceptions[i]); + } + } +}; + +} // namespace alp + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_analyze.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_analyze.hpp new file mode 100644 index 000000000..0288a9bd4 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_analyze.hpp @@ -0,0 +1,173 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_analyze.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/function/compression_function.hpp" +#include "duckdb/storage/compression/alp/algorithm/alp.hpp" +#include "duckdb/storage/compression/alp/alp_utils.hpp" +#include "duckdb/storage/compression/alp/alp_constants.hpp" +#include "duckdb/storage/compression/patas/patas.hpp" + +#include + +namespace duckdb { + +template +struct AlpAnalyzeState : public AnalyzeState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + AlpAnalyzeState() : state() { + } + + idx_t total_bytes_used = 0; + idx_t current_bytes_used_in_segment = 0; + idx_t vectors_sampled_count = 0; + idx_t total_values_count = 0; + idx_t vectors_count = 0; + vector> rowgroup_sample; + vector> complete_vectors_sampled; + alp::AlpCompressionState state; + +public: + // Returns the required space to hyphotetically store the compressed segment + void FlushSegment() { + // We add the size of the segment header (the pointer to the metadata) + total_bytes_used += current_bytes_used_in_segment + AlpConstants::METADATA_POINTER_SIZE; + current_bytes_used_in_segment = 0; + } + + // Returns the required space to hyphotetically store the compressed vector + idx_t RequiredSpace() const { + idx_t required_space = + state.bp_size + state.exceptions_count * (sizeof(EXACT_TYPE) + AlpConstants::EXCEPTION_POSITION_SIZE) + + AlpConstants::EXPONENT_SIZE + AlpConstants::FACTOR_SIZE + AlpConstants::EXCEPTIONS_COUNT_SIZE + + AlpConstants::FOR_SIZE + AlpConstants::BIT_WIDTH_SIZE + AlpConstants::METADATA_POINTER_SIZE; + return required_space; + } + + void FlushVector() { + current_bytes_used_in_segment += RequiredSpace(); + state.Reset(); + } + + // Check if we have enough space in the segment to hyphotetically store the compressed vector + bool HasEnoughSpace() { + idx_t bytes_to_be_used = AlignValue(current_bytes_used_in_segment + RequiredSpace()); + // We have enough space if the already used space + the required space for a new vector + // does not exceed the space of the block - the segment header (the pointer to the metadata) + return bytes_to_be_used <= (Storage::BLOCK_SIZE - AlpConstants::METADATA_POINTER_SIZE); + } + + idx_t TotalUsedBytes() const { + return AlignValue(total_bytes_used); + } +}; + +template +unique_ptr AlpInitAnalyze(ColumnData &col_data, PhysicalType type) { + return make_uniq>(); +} + +/* + * ALP Analyze step only pushes the needed samples to estimate the compression size in the finalize step + */ +template +bool AlpAnalyze(AnalyzeState &state, Vector &input, idx_t count) { + auto &analyze_state = (AlpAnalyzeState &)state; + bool must_skip_current_vector = alp::AlpUtils::MustSkipSamplingFromCurrentVector( + analyze_state.vectors_count, analyze_state.vectors_sampled_count, count); + analyze_state.vectors_count += 1; + analyze_state.total_values_count += count; + if (must_skip_current_vector) { + return true; + } + + UnifiedVectorFormat vdata; + input.ToUnifiedFormat(count, vdata); + auto data = UnifiedVectorFormat::GetData(vdata); + + alp::AlpSamplingParameters sampling_params = alp::AlpUtils::GetSamplingParameters(count); + + vector current_vector_null_positions(sampling_params.n_lookup_values, 0); + vector current_vector_values(sampling_params.n_lookup_values, 0); + vector current_vector_sample(sampling_params.n_sampled_values, 0); + + // Storing the entire sampled vector + //! We need to store the entire sampled vector to perform the 'analyze' compression in it + idx_t nulls_idx = 0; + // We optimize by doing a different loop when there are no nulls + if (vdata.validity.AllValid()) { + for (idx_t i = 0; i < sampling_params.n_lookup_values; i++) { + auto idx = vdata.sel->get_index(i); + T value = data[idx]; + current_vector_values[i] = value; + } + } else { + for (idx_t i = 0; i < sampling_params.n_lookup_values; i++) { + auto idx = vdata.sel->get_index(i); + T value = data[idx]; + //! We resolve null values with a predicated comparison + bool is_null = !vdata.validity.RowIsValid(idx); + current_vector_null_positions[nulls_idx] = i; + nulls_idx += is_null; + current_vector_values[i] = value; + } + alp::AlpUtils::FindAndReplaceNullsInVector(current_vector_values.data(), + current_vector_null_positions.data(), + sampling_params.n_lookup_values, nulls_idx); + } + + // Storing the sample of that vector + idx_t sample_idx = 0; + for (idx_t i = 0; i < sampling_params.n_lookup_values; i += sampling_params.n_sampled_increments) { + current_vector_sample[sample_idx] = current_vector_values[i]; + sample_idx++; + } + D_ASSERT(sample_idx == sampling_params.n_sampled_values); + + //! A std::move is needed to avoid a copy of the pushed vector + analyze_state.complete_vectors_sampled.push_back(std::move(current_vector_values)); + analyze_state.rowgroup_sample.push_back(std::move(current_vector_sample)); + analyze_state.vectors_sampled_count++; + return true; +} + +/* + * Estimate the compression size of ALP using the taken samples + */ +template +idx_t AlpFinalAnalyze(AnalyzeState &state) { + auto &analyze_state = (AlpAnalyzeState &)state; + + // Finding the Top K combinations of Exponent and Factor + alp::AlpCompression::FindTopKCombinations(analyze_state.rowgroup_sample, analyze_state.state); + + // Encode the entire sampled vectors to estimate a compression size + idx_t compressed_values = 0; + for (auto &vector_to_compress : analyze_state.complete_vectors_sampled) { + alp::AlpCompression::Compress(vector_to_compress.data(), vector_to_compress.size(), + analyze_state.state); + if (!analyze_state.HasEnoughSpace()) { + analyze_state.FlushSegment(); + } + analyze_state.FlushVector(); + compressed_values += vector_to_compress.size(); + } + + // Flush last unfinished segment + analyze_state.FlushSegment(); + + // We estimate the size by taking into account the portion of the values we took + const auto factor_of_sampling = analyze_state.total_values_count / compressed_values; + const auto final_analyze_size = analyze_state.TotalUsedBytes() * factor_of_sampling; + return final_analyze_size; // return size of data in bytes +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_compress.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_compress.hpp new file mode 100644 index 000000000..83b886207 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_compress.hpp @@ -0,0 +1,283 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_compress.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/patas/patas.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/storage/compression/alp/algorithm/alp.hpp" +#include "duckdb/storage/compression/alp/alp_analyze.hpp" + +#include "duckdb/common/helper.hpp" +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" + +#include + +namespace duckdb { + +template +struct AlpCompressionState : public CompressionState { + +public: + using EXACT_TYPE = typename FloatingToExact::type; + explicit AlpCompressionState(ColumnDataCheckpointer &checkpointer, AlpAnalyzeState *analyze_state) + : checkpointer(checkpointer), function(checkpointer.GetCompressionFunction(CompressionType::COMPRESSION_ALP)) { + CreateEmptySegment(checkpointer.GetRowGroup().start); + + //! Combinations found on the analyze step are needed for compression + state.best_k_combinations = analyze_state->state.best_k_combinations; + } + + ColumnDataCheckpointer &checkpointer; + CompressionFunction &function; + unique_ptr current_segment; + BufferHandle handle; + + idx_t vector_idx = 0; + idx_t nulls_idx = 0; + idx_t vectors_flushed = 0; + idx_t data_bytes_used = 0; + + data_ptr_t data_ptr; // Pointer to next free spot in segment; + data_ptr_t metadata_ptr; // Reverse pointer to the next free spot for the metadata; used in decoding to SKIP vectors + uint32_t next_vector_byte_index_start = AlpConstants::HEADER_SIZE; + + T input_vector[AlpConstants::ALP_VECTOR_SIZE]; + uint16_t vector_null_positions[AlpConstants::ALP_VECTOR_SIZE]; + + alp::AlpCompressionState state; + +public: + // Returns the space currently used in the segment (in bytes) + idx_t UsedSpace() const { + return AlpConstants::METADATA_POINTER_SIZE + data_bytes_used; + } + + // Returns the required space to store the newly compressed vector + idx_t RequiredSpace() { + idx_t required_space = + state.bp_size + (state.exceptions_count * (sizeof(EXACT_TYPE) + AlpConstants::EXCEPTION_POSITION_SIZE)) + + AlpConstants::EXPONENT_SIZE + AlpConstants::FACTOR_SIZE + AlpConstants::EXCEPTIONS_COUNT_SIZE + + AlpConstants::FOR_SIZE + AlpConstants::BIT_WIDTH_SIZE; + return required_space; + } + + bool HasEnoughSpace() { + //! If [start of block + used space + required space] is more than whats left (current position + //! of metadata pointer - the size of a new metadata pointer) + if ((handle.Ptr() + AlignValue(UsedSpace() + RequiredSpace())) >= + (metadata_ptr - AlpConstants::METADATA_POINTER_SIZE)) { + return false; + } + return true; + } + + void ResetVector() { + state.Reset(); + } + + void CreateEmptySegment(idx_t row_start) { + auto &db = checkpointer.GetDatabase(); + auto &type = checkpointer.GetType(); + auto compressed_segment = ColumnSegment::CreateTransientSegment(db, type, row_start); + current_segment = std::move(compressed_segment); + current_segment->function = function; + auto &buffer_manager = BufferManager::GetBufferManager(current_segment->db); + handle = buffer_manager.Pin(current_segment->block); + + // Pointer to the start of the compressed data + data_ptr = handle.Ptr() + current_segment->GetBlockOffset() + AlpConstants::HEADER_SIZE; + // Pointer to the start of the Metadata + metadata_ptr = handle.Ptr() + current_segment->GetBlockOffset() + Storage::BLOCK_SIZE; + + next_vector_byte_index_start = AlpConstants::HEADER_SIZE; + } + + void CompressVector() { + if (nulls_idx) { + alp::AlpUtils::FindAndReplaceNullsInVector(input_vector, vector_null_positions, vector_idx, nulls_idx); + } + alp::AlpCompression::Compress(input_vector, vector_idx, vector_null_positions, nulls_idx, state); + //! Check if the compressed vector fits on current segment + if (!HasEnoughSpace()) { + auto row_start = current_segment->start + current_segment->count; + FlushSegment(); + CreateEmptySegment(row_start); + } + + if (vector_idx != nulls_idx) { //! At least there is one valid value in the vector + for (idx_t i = 0; i < vector_idx; i++) { + NumericStats::Update(current_segment->stats.statistics, input_vector[i]); + } + } + current_segment->count += vector_idx; + FlushVector(); + } + + // Stores the vector and its metadata + void FlushVector() { + Store(state.vector_encoding_indices.exponent, data_ptr); + data_ptr += AlpConstants::EXPONENT_SIZE; + + Store(state.vector_encoding_indices.factor, data_ptr); + data_ptr += AlpConstants::FACTOR_SIZE; + + Store(state.exceptions_count, data_ptr); + data_ptr += AlpConstants::EXCEPTIONS_COUNT_SIZE; + + Store(state.frame_of_reference, data_ptr); + data_ptr += AlpConstants::FOR_SIZE; + + Store(state.bit_width, data_ptr); + data_ptr += AlpConstants::BIT_WIDTH_SIZE; + + memcpy((void *)data_ptr, (void *)state.values_encoded, state.bp_size); + // We should never go out of bounds in the values_encoded array + D_ASSERT((AlpConstants::ALP_VECTOR_SIZE * 8) >= state.bp_size); + + data_ptr += state.bp_size; + + if (state.exceptions_count > 0) { + memcpy((void *)data_ptr, (void *)state.exceptions, sizeof(EXACT_TYPE) * state.exceptions_count); + data_ptr += sizeof(EXACT_TYPE) * state.exceptions_count; + memcpy((void *)data_ptr, (void *)state.exceptions_positions, + AlpConstants::EXCEPTION_POSITION_SIZE * state.exceptions_count); + data_ptr += AlpConstants::EXCEPTION_POSITION_SIZE * state.exceptions_count; + } + + data_bytes_used += state.bp_size + + (state.exceptions_count * (sizeof(EXACT_TYPE) + AlpConstants::EXCEPTION_POSITION_SIZE)) + + AlpConstants::EXPONENT_SIZE + AlpConstants::FACTOR_SIZE + + AlpConstants::EXCEPTIONS_COUNT_SIZE + AlpConstants::FOR_SIZE + AlpConstants::BIT_WIDTH_SIZE; + + // Write pointer to the vector data (metadata) + metadata_ptr -= sizeof(uint32_t); + Store(next_vector_byte_index_start, metadata_ptr); + next_vector_byte_index_start = UsedSpace(); + + vectors_flushed++; + vector_idx = 0; + nulls_idx = 0; + ResetVector(); + } + + void FlushSegment() { + auto &checkpoint_state = checkpointer.GetCheckpointState(); + auto dataptr = handle.Ptr(); + + idx_t metadata_offset = AlignValue(UsedSpace()); + + // Verify that the metadata_ptr is not smaller than the space used by the data + D_ASSERT(dataptr + metadata_offset <= metadata_ptr); + + idx_t bytes_used_by_metadata = dataptr + Storage::BLOCK_SIZE - metadata_ptr; + + // Initially the total segment size is the size of the block + idx_t total_segment_size = Storage::BLOCK_SIZE; + + //! We compact the block if the space used is less than a threshold + const auto used_space_percentage = + static_cast(metadata_offset + bytes_used_by_metadata) / static_cast(total_segment_size); + if (used_space_percentage < AlpConstants::COMPACT_BLOCK_THRESHOLD) { +#ifdef DEBUG + //! Copy the first 4 bytes of the metadata + uint32_t verify_bytes; + memcpy((void *)&verify_bytes, metadata_ptr, 4); +#endif + memmove(dataptr + metadata_offset, metadata_ptr, bytes_used_by_metadata); +#ifdef DEBUG + //! Now assert that the memmove was correct + D_ASSERT(verify_bytes == *(uint32_t *)(dataptr + metadata_offset)); +#endif + total_segment_size = metadata_offset + bytes_used_by_metadata; + } + + // Store the offset to the end of metadata (to be used as a backwards pointer in decoding) + Store(total_segment_size, dataptr); + + handle.Destroy(); + checkpoint_state.FlushSegment(std::move(current_segment), total_segment_size); + data_bytes_used = 0; + vectors_flushed = 0; + } + + void Finalize() { + if (vector_idx != 0) { + CompressVector(); + D_ASSERT(vector_idx == 0); + } + FlushSegment(); + current_segment.reset(); + } + + void Append(UnifiedVectorFormat &vdata, idx_t count) { + auto data = UnifiedVectorFormat::GetData(vdata); + idx_t values_left_in_data = count; + idx_t offset_in_data = 0; + while (values_left_in_data > 0) { + // We calculate until which value in data we must go to fill the input_vector + // to avoid checking if input_vector is filled in each iteration + auto values_to_fill_alp_input = + MinValue(AlpConstants::ALP_VECTOR_SIZE - vector_idx, values_left_in_data); + if (vdata.validity.AllValid()) { //! We optimize a loop when there are no null + for (idx_t i = 0; i < values_to_fill_alp_input; i++) { + auto idx = vdata.sel->get_index(offset_in_data + i); + T value = data[idx]; + input_vector[vector_idx + i] = value; + } + } else { + for (idx_t i = 0; i < values_to_fill_alp_input; i++) { + auto idx = vdata.sel->get_index(offset_in_data + i); + T value = data[idx]; + bool is_null = !vdata.validity.RowIsValid(idx); + //! We resolve null values with a predicated comparison + vector_null_positions[nulls_idx] = vector_idx + i; + nulls_idx += is_null; + input_vector[vector_idx + i] = value; + } + } + offset_in_data += values_to_fill_alp_input; + values_left_in_data -= values_to_fill_alp_input; + vector_idx += values_to_fill_alp_input; + // We still need this check since we could have an incomplete input_vector at the end of data + if (vector_idx == AlpConstants::ALP_VECTOR_SIZE) { + CompressVector(); + D_ASSERT(vector_idx == 0); + } + } + } +}; + +template +unique_ptr AlpInitCompression(ColumnDataCheckpointer &checkpointer, unique_ptr state) { + return make_uniq>(checkpointer, (AlpAnalyzeState *)state.get()); +} + +template +void AlpCompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { + auto &state = (AlpCompressionState &)state_p; + UnifiedVectorFormat vdata; + scan_vector.ToUnifiedFormat(count, vdata); + state.Append(vdata, count); +} + +template +void AlpFinalizeCompress(CompressionState &state_p) { + auto &state = (AlpCompressionState &)state_p; + state.Finalize(); +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_constants.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_constants.hpp new file mode 100644 index 000000000..55353dda1 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_constants.hpp @@ -0,0 +1,134 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_constants.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/limits.hpp" +#include "duckdb/main/config.hpp" + +namespace duckdb { + +class AlpConstants { +public: + static constexpr uint32_t ALP_VECTOR_SIZE = 1024; + static constexpr uint32_t RG_SAMPLES = 8; + static constexpr uint16_t SAMPLES_PER_VECTOR = 32; + // We calculate how many equidistant vector we must jump within a rowgroup + static constexpr uint32_t RG_SAMPLES_DUCKDB_JUMP = (STANDARD_ROW_GROUPS_SIZE / RG_SAMPLES) / STANDARD_VECTOR_SIZE; + + static constexpr uint8_t HEADER_SIZE = sizeof(uint32_t); + static constexpr uint8_t EXPONENT_SIZE = sizeof(uint8_t); + static constexpr uint8_t FACTOR_SIZE = sizeof(uint8_t); + static constexpr uint8_t EXCEPTIONS_COUNT_SIZE = sizeof(uint16_t); + static constexpr uint8_t EXCEPTION_POSITION_SIZE = sizeof(uint16_t); + static constexpr uint8_t FOR_SIZE = sizeof(uint64_t); + static constexpr uint8_t BIT_WIDTH_SIZE = sizeof(uint8_t); + static constexpr uint8_t METADATA_POINTER_SIZE = sizeof(uint32_t); + + static constexpr uint8_t SAMPLING_EARLY_EXIT_THRESHOLD = 2; + + static constexpr double COMPACT_BLOCK_THRESHOLD = 0.80; + + // Largest double which fits into an int64 + static constexpr double ENCODING_UPPER_LIMIT = 9223372036854774784; + static constexpr double ENCODING_LOWER_LIMIT = -9223372036854774784; + + static constexpr uint8_t MAX_COMBINATIONS = 5; + + static constexpr const int64_t FACT_ARR[] = {1, + 10, + 100, + 1000, + 10000, + 100000, + 1000000, + 10000000, + 100000000, + 1000000000, + 10000000000, + 100000000000, + 1000000000000, + 10000000000000, + 100000000000000, + 1000000000000000, + 10000000000000000, + 100000000000000000, + 1000000000000000000}; +}; + +template +struct AlpTypedConstants {}; + +template <> +struct AlpTypedConstants { + + static constexpr float MAGIC_NUMBER = 12582912.0; //! 2^22 + 2^23 + static constexpr uint8_t MAX_EXPONENT = 10; + + static constexpr const float EXP_ARR[] = {1.0, 10.0, 100.0, 1000.0, 10000.0, 100000.0, + 1000000.0, 10000000.0, 100000000.0, 1000000000.0, 10000000000.0}; + + static constexpr float FRAC_ARR[] = {1.0, 0.1, 0.01, 0.001, 0.0001, 0.00001, + 0.000001, 0.0000001, 0.00000001, 0.000000001, 0.0000000001}; +}; + +template <> +struct AlpTypedConstants { + + static constexpr double MAGIC_NUMBER = 6755399441055744.0; //! 2^51 + 2^52 + static constexpr uint8_t MAX_EXPONENT = 18; //! 10^18 is the maximum int64 + + static constexpr const double EXP_ARR[] = {1.0, + 10.0, + 100.0, + 1000.0, + 10000.0, + 100000.0, + 1000000.0, + 10000000.0, + 100000000.0, + 1000000000.0, + 10000000000.0, + 100000000000.0, + 1000000000000.0, + 10000000000000.0, + 100000000000000.0, + 1000000000000000.0, + 10000000000000000.0, + 100000000000000000.0, + 1000000000000000000.0, + 10000000000000000000.0, + 100000000000000000000.0, + 1000000000000000000000.0, + 10000000000000000000000.0, + 100000000000000000000000.0}; + + static constexpr double FRAC_ARR[] = {1.0, + 0.1, + 0.01, + 0.001, + 0.0001, + 0.00001, + 0.000001, + 0.0000001, + 0.00000001, + 0.000000001, + 0.0000000001, + 0.00000000001, + 0.000000000001, + 0.0000000000001, + 0.00000000000001, + 0.000000000000001, + 0.0000000000000001, + 0.00000000000000001, + 0.000000000000000001, + 0.0000000000000000001, + 0.00000000000000000001}; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_fetch.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_fetch.hpp new file mode 100644 index 000000000..208c85a3d --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_fetch.hpp @@ -0,0 +1,42 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_fetch.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/alp/alp_scan.hpp" + +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" + +namespace duckdb { + +template +void AlpFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t row_id, Vector &result, idx_t result_idx) { + using EXACT_TYPE = typename FloatingToExact::type; + + AlpScanState scan_state(segment); + scan_state.Skip(segment, row_id); + auto result_data = FlatVector::GetData(result); + result_data[result_idx] = (EXACT_TYPE)0; + + if (scan_state.VectorFinished() && scan_state.total_value_count < scan_state.count) { + scan_state.LoadVector(scan_state.vector_state.decoded_values); + } + scan_state.vector_state.Scan((uint8_t *)(result_data + result_idx), 1); + scan_state.total_value_count++; +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_scan.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_scan.hpp new file mode 100644 index 000000000..93f508f4d --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_scan.hpp @@ -0,0 +1,244 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_scan.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/alp/algorithm/alp.hpp" + +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" +#include "duckdb/storage/table/scan_state.hpp" + +namespace duckdb { + +template +struct AlpVectorState { +public: + void Reset() { + index = 0; + } + + // Scan of the data itself + template + void Scan(uint8_t *dest, idx_t count) { + if (!SKIP) { + memcpy(dest, (void *)(decoded_values + index), sizeof(T) * count); + } + index += count; + } + + template + void LoadValues(T *value_buffer, idx_t count) { + if (SKIP) { + return; + } + value_buffer[0] = (T)0; + alp::AlpDecompression::Decompress(for_encoded, value_buffer, count, v_factor, v_exponent, exceptions_count, + exceptions, exceptions_positions, frame_of_reference, bit_width); + } + +public: + idx_t index; + T decoded_values[AlpConstants::ALP_VECTOR_SIZE]; + T exceptions[AlpConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions_positions[AlpConstants::ALP_VECTOR_SIZE]; + uint8_t for_encoded[AlpConstants::ALP_VECTOR_SIZE * 8]; + uint8_t v_exponent; + uint8_t v_factor; + uint16_t exceptions_count; + uint64_t frame_of_reference; + uint8_t bit_width; +}; + +template +struct AlpScanState : public SegmentScanState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + explicit AlpScanState(ColumnSegment &segment) : segment(segment), count(segment.count) { + auto &buffer_manager = BufferManager::GetBufferManager(segment.db); + handle = buffer_manager.Pin(segment.block); + // ScanStates never exceed the boundaries of a Segment, + // but are not guaranteed to start at the beginning of the Block + segment_data = handle.Ptr() + segment.GetBlockOffset(); + auto metadata_offset = Load(segment_data); + metadata_ptr = segment_data + metadata_offset; + } + + BufferHandle handle; + data_ptr_t metadata_ptr; + data_ptr_t segment_data; + idx_t total_value_count = 0; + AlpVectorState vector_state; + + ColumnSegment &segment; + idx_t count; + + idx_t LeftInVector() const { + return AlpConstants::ALP_VECTOR_SIZE - (total_value_count % AlpConstants::ALP_VECTOR_SIZE); + } + + inline bool VectorFinished() const { + return (total_value_count % AlpConstants::ALP_VECTOR_SIZE) == 0; + } + + // Scan up to a vector boundary + template + void ScanVector(T *values, idx_t vector_size) { + D_ASSERT(vector_size <= AlpConstants::ALP_VECTOR_SIZE); + D_ASSERT(vector_size <= LeftInVector()); + if (VectorFinished() && total_value_count < count) { + if (vector_size == AlpConstants::ALP_VECTOR_SIZE) { + LoadVector(values); + total_value_count += vector_size; + return; + } else { + // Even if SKIP is given, the vector size is not big enough to be able to fully skip the entire vector + LoadVector(vector_state.decoded_values); + } + } + vector_state.template Scan((uint8_t *)values, vector_size); + + total_value_count += vector_size; + } + + // Using the metadata, we can avoid loading any of the data if we don't care about the vector at all + void SkipVector() { + // Skip the offset indicating where the data starts + metadata_ptr -= AlpConstants::METADATA_POINTER_SIZE; + idx_t vector_size = MinValue((idx_t)AlpConstants::ALP_VECTOR_SIZE, count - total_value_count); + total_value_count += vector_size; + } + + template + void LoadVector(T *value_buffer) { + vector_state.Reset(); + + // Load the offset (metadata) indicating where the vector data starts + metadata_ptr -= AlpConstants::METADATA_POINTER_SIZE; + auto data_byte_offset = Load(metadata_ptr); + D_ASSERT(data_byte_offset < Storage::BLOCK_SIZE); + + idx_t vector_size = MinValue((idx_t)AlpConstants::ALP_VECTOR_SIZE, (count - total_value_count)); + + data_ptr_t vector_ptr = segment_data + data_byte_offset; + + // Load the vector data + vector_state.v_exponent = Load(vector_ptr); + vector_ptr += AlpConstants::EXPONENT_SIZE; + + vector_state.v_factor = Load(vector_ptr); + vector_ptr += AlpConstants::FACTOR_SIZE; + + vector_state.exceptions_count = Load(vector_ptr); + vector_ptr += AlpConstants::EXCEPTIONS_COUNT_SIZE; + + vector_state.frame_of_reference = Load(vector_ptr); + vector_ptr += AlpConstants::FOR_SIZE; + + vector_state.bit_width = Load(vector_ptr); + vector_ptr += AlpConstants::BIT_WIDTH_SIZE; + + D_ASSERT(vector_state.exceptions_count <= vector_size); + D_ASSERT(vector_state.v_exponent <= AlpTypedConstants::MAX_EXPONENT); + D_ASSERT(vector_state.v_factor <= vector_state.v_exponent); + D_ASSERT(vector_state.bit_width <= sizeof(uint64_t) * 8); + + if (vector_state.bit_width > 0) { + auto bp_size = BitpackingPrimitives::GetRequiredSize(vector_size, vector_state.bit_width); + memcpy(vector_state.for_encoded, (void *)vector_ptr, bp_size); + vector_ptr += bp_size; + } + + if (vector_state.exceptions_count > 0) { + memcpy(vector_state.exceptions, (void *)vector_ptr, sizeof(EXACT_TYPE) * vector_state.exceptions_count); + vector_ptr += sizeof(EXACT_TYPE) * vector_state.exceptions_count; + memcpy(vector_state.exceptions_positions, (void *)vector_ptr, + AlpConstants::EXCEPTION_POSITION_SIZE * vector_state.exceptions_count); + } + + // Decode all the vector values to the specified 'value_buffer' + vector_state.template LoadValues(value_buffer, vector_size); + } + +public: + //! Skip the next 'skip_count' values, we don't store the values + void Skip(ColumnSegment &col_segment, idx_t skip_count) { + + if (total_value_count != 0 && !VectorFinished()) { + // Finish skipping the current vector + idx_t to_skip = LeftInVector(); + skip_count -= to_skip; + ScanVector(nullptr, to_skip); + } + // Figure out how many entire vectors we can skip + // For these vectors, we don't even need to process the metadata or values + idx_t vectors_to_skip = skip_count / AlpConstants::ALP_VECTOR_SIZE; + for (idx_t i = 0; i < vectors_to_skip; i++) { + SkipVector(); + } + skip_count -= AlpConstants::ALP_VECTOR_SIZE * vectors_to_skip; + if (skip_count == 0) { + return; + } + // For the last vector that this skip (partially) touches, we do need to + // load the metadata and values into the vector_state because + // we don't know exactly how many they are + ScanVector(nullptr, skip_count); + } +}; + +template +unique_ptr AlpInitScan(ColumnSegment &segment) { + auto result = make_uniq_base>(segment); + return result; +} + +//===--------------------------------------------------------------------===// +// Scan base data +//===--------------------------------------------------------------------===// +template +void AlpScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result, + idx_t result_offset) { + auto &scan_state = (AlpScanState &)*state.scan_state; + + // Get the pointer to the result values + auto current_result_ptr = FlatVector::GetData(result); + result.SetVectorType(VectorType::FLAT_VECTOR); + current_result_ptr += result_offset; + + idx_t scanned = 0; + while (scanned < scan_count) { + const auto remaining = scan_count - scanned; + const idx_t to_scan = MinValue(remaining, scan_state.LeftInVector()); + + scan_state.template ScanVector(current_result_ptr + scanned, to_scan); + scanned += to_scan; + } +} + +template +void AlpSkip(ColumnSegment &segment, ColumnScanState &state, idx_t skip_count) { + auto &scan_state = (AlpScanState &)*state.scan_state; + scan_state.Skip(segment, skip_count); +} + +template +void AlpScan(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result) { + AlpScanPartial(segment, state, scan_count, result, 0); +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alp/alp_utils.hpp b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_utils.hpp new file mode 100644 index 000000000..dd6b566ad --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alp/alp_utils.hpp @@ -0,0 +1,103 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_utils.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/function/compression_function.hpp" +#include "duckdb/storage/compression/alp/alp_constants.hpp" +#include "duckdb/storage/compression/patas/patas.hpp" + +#include + +namespace duckdb { + +namespace alp { + +struct AlpSamplingParameters { + uint32_t n_lookup_values; + uint32_t n_sampled_increments; + uint32_t n_sampled_values; + + AlpSamplingParameters(uint32_t n_lookup_values, uint32_t n_sampled_increments, uint32_t n_sampled_values) + : n_lookup_values(n_lookup_values), n_sampled_increments(n_sampled_increments), + n_sampled_values(n_sampled_values) { + } +}; + +class AlpUtils { +public: + AlpUtils() { + } + +public: + static AlpSamplingParameters GetSamplingParameters(idx_t current_vector_n_values) { + + uint32_t n_lookup_values = MinValue(current_vector_n_values, (idx_t)AlpConstants::ALP_VECTOR_SIZE); + //! We sample equidistant values within a vector; to do this we jump a fixed number of values + uint32_t n_sampled_increments = + MaxValue(1, (int32_t)std::ceil((double)n_lookup_values / AlpConstants::SAMPLES_PER_VECTOR)); + uint32_t n_sampled_values = std::ceil((double)n_lookup_values / n_sampled_increments); + D_ASSERT(n_sampled_values < AlpConstants::ALP_VECTOR_SIZE); + + AlpSamplingParameters sampling_params = {n_lookup_values, n_sampled_increments, n_sampled_values}; + return sampling_params; + } + + static bool MustSkipSamplingFromCurrentVector(idx_t vectors_count, idx_t vectors_sampled_count, + idx_t current_vector_n_values) { + //! We sample equidistant vectors; to do this we skip a fixed values of vectors + bool must_select_rowgroup_samples = (vectors_count % AlpConstants::RG_SAMPLES_DUCKDB_JUMP) == 0; + + //! If we are not in the correct jump, we do not take sample from this vector + if (!must_select_rowgroup_samples) { + return true; + } + + //! We do not take samples of non-complete duckdb vectors (usually the last one) + //! Except in the case of too little data + if (current_vector_n_values < AlpConstants::SAMPLES_PER_VECTOR && vectors_sampled_count != 0) { + return true; + } + return false; + } + + template + static T FindFirstValueNotInPositionsArray(const T *input_vector, const uint16_t *positions, idx_t values_count) { + T a_non_special_value = 0; + for (idx_t i = 0; i < values_count; i++) { + if (i != positions[i]) { + a_non_special_value = input_vector[i]; + break; + } + } + return a_non_special_value; + } + + template + static void ReplaceValueInVectorPositions(T *input_vector, const uint16_t *positions_to_replace, + idx_t special_values_count, T value_to_replace) { + for (idx_t i = 0; i < special_values_count; i++) { + uint16_t null_value_pos = positions_to_replace[i]; + input_vector[null_value_pos] = value_to_replace; + } + } + + template + static void FindAndReplaceNullsInVector(T *input_vector, const uint16_t *vector_null_positions, idx_t values_count, + idx_t nulls_count) { + if (nulls_count == 0) { + return; + } + T a_non_null_value = FindFirstValueNotInPositionsArray(input_vector, vector_null_positions, values_count); + ReplaceValueInVectorPositions(input_vector, vector_null_positions, nulls_count, a_non_null_value); + } +}; + +} // namespace alp + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/algorithm/alprd.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/algorithm/alprd.hpp new file mode 100644 index 000000000..cee6e20bd --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/algorithm/alprd.hpp @@ -0,0 +1,235 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alprd/algorithm/alprd.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/patas/patas.hpp" +#include "duckdb/storage/compression/alprd/alprd_constants.hpp" + +#include "duckdb/common/bitpacking.hpp" +#include "duckdb/common/common.hpp" +#include "duckdb/common/pair.hpp" +#include "duckdb/common/unordered_map.hpp" +#include "duckdb/common/limits.hpp" + +#include + +namespace duckdb { + +namespace alp { + +struct AlpRDLeftPartInfo { + uint32_t count; + uint64_t hash; + AlpRDLeftPartInfo(uint32_t count, uint64_t hash) : count(count), hash(hash) { + } +}; + +template +class AlpRDCompressionState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + AlpRDCompressionState() : right_bit_width(0), left_bit_width(0), exceptions_count(0) { + } + + void Reset() { + left_bit_packed_size = 0; + right_bit_packed_size = 0; + exceptions_count = 0; + } + +public: + uint8_t right_bit_width; // 'right' & 'left' refer to the respective parts of the floating numbers after splitting + uint8_t left_bit_width; + uint16_t exceptions_count; + uint8_t right_parts_encoded[AlpRDConstants::ALP_VECTOR_SIZE * 8]; + uint8_t left_parts_encoded[AlpRDConstants::ALP_VECTOR_SIZE * 8]; + uint16_t left_parts_dict[AlpRDConstants::MAX_DICTIONARY_SIZE]; + uint16_t exceptions[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions_positions[AlpRDConstants::ALP_VECTOR_SIZE]; + idx_t left_bit_packed_size; + idx_t right_bit_packed_size; + unordered_map left_parts_dict_map; + uint8_t actual_dictionary_size; +}; + +template +struct AlpRDCompression { + using State = AlpRDCompressionState; + using EXACT_TYPE = typename FloatingToExact::type; + static constexpr uint8_t EXACT_TYPE_BITSIZE = sizeof(EXACT_TYPE) * 8; + + /* + * Estimate the bits per value of ALPRD within a sample + */ + static double EstimateCompressionSize(uint8_t right_bit_width, uint8_t left_bit_width, uint16_t exceptions_count, + uint64_t sample_count) { + double exceptions_size = + exceptions_count * ((AlpRDConstants::EXCEPTION_POSITION_SIZE + AlpRDConstants::EXCEPTION_SIZE) * 8); + double estimated_size = right_bit_width + left_bit_width + (exceptions_size / sample_count); + return estimated_size; + } + + template + static double BuildLeftPartsDictionary(const vector &values, uint8_t right_bit_width, State &state) { + unordered_map left_parts_hash; + vector left_parts_sorted_repetitions; + + // Building a hash for all the left parts and how many times they appear + for (auto &value : values) { + auto left_tmp = value >> right_bit_width; + left_parts_hash[left_tmp]++; + } + + // We build a vector from the hash to be able to sort it by repetition count + left_parts_sorted_repetitions.reserve(left_parts_hash.size()); + for (auto &hash_pair : left_parts_hash) { + left_parts_sorted_repetitions.emplace_back(hash_pair.second, hash_pair.first); + } + sort(left_parts_sorted_repetitions.begin(), left_parts_sorted_repetitions.end(), + [](const AlpRDLeftPartInfo &a, const AlpRDLeftPartInfo &b) { return a.count > b.count; }); + + // Exceptions are left parts which do not fit in the fixed dictionary size + uint32_t exceptions_count = 0; + for (idx_t i = AlpRDConstants::MAX_DICTIONARY_SIZE; i < left_parts_sorted_repetitions.size(); i++) { + exceptions_count += left_parts_sorted_repetitions[i].count; + } + + // The left parts bit width after compression is determined by how many elements are in the dictionary + uint64_t actual_dictionary_size = + MinValue(AlpRDConstants::MAX_DICTIONARY_SIZE, left_parts_sorted_repetitions.size()); + uint8_t left_bit_width = MaxValue(1, std::ceil(std::log2(actual_dictionary_size))); + + if (PERSIST_DICT) { + for (idx_t dict_idx = 0; dict_idx < actual_dictionary_size; dict_idx++) { + //! The dict keys are mapped to the left part themselves + state.left_parts_dict[dict_idx] = left_parts_sorted_repetitions[dict_idx].hash; + state.left_parts_dict_map.insert({state.left_parts_dict[dict_idx], dict_idx}); + } + //! Pararelly we store a map of the dictionary to quickly resolve exceptions during encoding + for (idx_t i = actual_dictionary_size + 1; i < left_parts_sorted_repetitions.size(); i++) { + state.left_parts_dict_map.insert({left_parts_sorted_repetitions[i].hash, i}); + } + state.left_bit_width = left_bit_width; + state.right_bit_width = right_bit_width; + state.actual_dictionary_size = actual_dictionary_size; + + D_ASSERT(state.left_bit_width > 0 && state.right_bit_width > 0 && + state.left_bit_width <= AlpRDConstants::MAX_DICTIONARY_BIT_WIDTH && + state.actual_dictionary_size <= AlpRDConstants::MAX_DICTIONARY_SIZE); + } + + double estimated_size = + EstimateCompressionSize(right_bit_width, left_bit_width, exceptions_count, values.size()); + return estimated_size; + } + + static double FindBestDictionary(const vector &values, State &state) { + uint8_t right_bit_width; + double best_dict_size = NumericLimits::Maximum(); + //! Finding the best position to CUT the values + for (idx_t i = 1; i <= AlpRDConstants::CUTTING_LIMIT; i++) { + uint8_t candidate_right_bit_width = EXACT_TYPE_BITSIZE - i; + double estimated_size = BuildLeftPartsDictionary(values, candidate_right_bit_width, state); + if (estimated_size <= best_dict_size) { + right_bit_width = candidate_right_bit_width; + best_dict_size = estimated_size; + } + // TODO: We could implement an early exit mechanism similar to normal ALP + } + double estimated_size = BuildLeftPartsDictionary(values, right_bit_width, state); + return estimated_size; + } + + static void Compress(const EXACT_TYPE *input_vector, idx_t n_values, State &state) { + + uint64_t right_parts[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t left_parts[AlpRDConstants::ALP_VECTOR_SIZE]; + + // Cutting the floating point values + for (idx_t i = 0; i < n_values; i++) { + EXACT_TYPE tmp = input_vector[i]; + right_parts[i] = tmp & ((1ULL << state.right_bit_width) - 1); + left_parts[i] = (tmp >> state.right_bit_width); + } + + // Dictionary encoding for left parts + for (idx_t i = 0; i < n_values; i++) { + uint16_t dictionary_index; + auto dictionary_key = left_parts[i]; + if (state.left_parts_dict_map.find(dictionary_key) == state.left_parts_dict_map.end()) { + //! If not found on the dictionary we store the smallest non-key index as exception (the dict size) + dictionary_index = state.actual_dictionary_size; + } else { + dictionary_index = state.left_parts_dict_map[dictionary_key]; + } + left_parts[i] = dictionary_index; + + //! Left parts not found in the dictionary are stored as exceptions + if (dictionary_index >= state.actual_dictionary_size) { + state.exceptions[state.exceptions_count] = dictionary_key; + state.exceptions_positions[state.exceptions_count] = i; + state.exceptions_count++; + } + } + + auto right_bit_packed_size = BitpackingPrimitives::GetRequiredSize(n_values, state.right_bit_width); + auto left_bit_packed_size = BitpackingPrimitives::GetRequiredSize(n_values, state.left_bit_width); + + if (!EMPTY) { + // Bitpacking Left and Right parts + BitpackingPrimitives::PackBuffer(state.left_parts_encoded, left_parts, n_values, + state.left_bit_width); + BitpackingPrimitives::PackBuffer(state.right_parts_encoded, right_parts, n_values, + state.right_bit_width); + } + + state.left_bit_packed_size = left_bit_packed_size; + state.right_bit_packed_size = right_bit_packed_size; + } +}; + +template +struct AlpRDDecompression { + using EXACT_TYPE = typename FloatingToExact::type; + + static void Decompress(uint8_t *left_encoded, uint8_t *right_encoded, const uint16_t *left_parts_dict, + EXACT_TYPE *output, idx_t values_count, uint16_t exceptions_count, + const uint16_t *exceptions, const uint16_t *exceptions_positions, uint8_t left_bit_width, + uint8_t right_bit_width) { + + uint8_t left_decoded[AlpRDConstants::ALP_VECTOR_SIZE * 8] = {0}; + uint8_t right_decoded[AlpRDConstants::ALP_VECTOR_SIZE * 8] = {0}; + + // Bitunpacking left and right parts + BitpackingPrimitives::UnPackBuffer(left_decoded, left_encoded, values_count, left_bit_width); + BitpackingPrimitives::UnPackBuffer(right_decoded, right_encoded, values_count, right_bit_width); + + uint16_t *left_parts = reinterpret_cast(data_ptr_cast(left_decoded)); + EXACT_TYPE *right_parts = reinterpret_cast(data_ptr_cast(right_decoded)); + + // Decoding + for (idx_t i = 0; i < values_count; i++) { + uint16_t left = left_parts_dict[left_parts[i]]; + EXACT_TYPE right = right_parts[i]; + output[i] = (static_cast(left) << right_bit_width) | right; + } + + // Exceptions Patching (exceptions only occur in left parts) + for (idx_t i = 0; i < exceptions_count; i++) { + EXACT_TYPE right = right_parts[exceptions_positions[i]]; + uint16_t left = exceptions[i]; + output[exceptions_positions[i]] = (static_cast(left) << right_bit_width) | right; + } + } +}; + +} // namespace alp + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_analyze.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_analyze.hpp new file mode 100644 index 000000000..6a2522ba6 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_analyze.hpp @@ -0,0 +1,134 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alprd/alprd_analyze.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/function/compression_function.hpp" +#include "duckdb/storage/compression/patas/patas.hpp" +#include "duckdb/storage/compression/alprd/algorithm/alprd.hpp" +#include "duckdb/storage/compression/alprd/alprd_constants.hpp" +#include "duckdb/storage/compression/alp/alp_utils.hpp" +#include "duckdb/storage/compression/alp/alp_constants.hpp" + +#include + +namespace duckdb { + +template +struct AlpRDAnalyzeState : public AnalyzeState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + AlpRDAnalyzeState() : state() { + } + + idx_t vectors_count = 0; + idx_t total_values_count = 0; + idx_t vectors_sampled_count = 0; + vector rowgroup_sample; + alp::AlpRDCompressionState state; +}; + +template +unique_ptr AlpRDInitAnalyze(ColumnData &col_data, PhysicalType type) { + return make_uniq>(); +} + +/* + * ALPRD Analyze step only pushes the needed samples to estimate the compression size in the finalize step + */ +template +bool AlpRDAnalyze(AnalyzeState &state, Vector &input, idx_t count) { + using EXACT_TYPE = typename FloatingToExact::type; + auto &analyze_state = (AlpRDAnalyzeState &)state; + + bool must_skip_current_vector = alp::AlpUtils::MustSkipSamplingFromCurrentVector( + analyze_state.vectors_count, analyze_state.vectors_sampled_count, count); + analyze_state.vectors_count += 1; + analyze_state.total_values_count += count; + if (must_skip_current_vector) { + return true; + } + + UnifiedVectorFormat vdata; + input.ToUnifiedFormat(count, vdata); + auto data = UnifiedVectorFormat::GetData(vdata); + + alp::AlpSamplingParameters sampling_params = alp::AlpUtils::GetSamplingParameters(count); + + vector current_vector_null_positions(sampling_params.n_lookup_values, 0); + vector current_vector_sample(sampling_params.n_sampled_values, 0); + + // Storing the sample of that vector + idx_t sample_idx = 0; + idx_t nulls_idx = 0; + // We optimize by doing a different loop when there are no nulls + if (vdata.validity.AllValid()) { + for (idx_t i = 0; i < sampling_params.n_lookup_values; i += sampling_params.n_sampled_increments) { + auto idx = vdata.sel->get_index(i); + EXACT_TYPE value = Load(const_data_ptr_cast(&data[idx])); + current_vector_sample[sample_idx] = value; + sample_idx++; + } + } else { + for (idx_t i = 0; i < sampling_params.n_lookup_values; i += sampling_params.n_sampled_increments) { + auto idx = vdata.sel->get_index(i); + EXACT_TYPE value = Load(const_data_ptr_cast(&data[idx])); + current_vector_sample[sample_idx] = value; + //! We resolve null values with a predicated comparison + bool is_null = !vdata.validity.RowIsValid(idx); + current_vector_null_positions[nulls_idx] = sample_idx; + nulls_idx += is_null; + sample_idx++; + } + alp::AlpUtils::FindAndReplaceNullsInVector(current_vector_sample.data(), + current_vector_null_positions.data(), + sampling_params.n_sampled_values, nulls_idx); + } + + D_ASSERT(sample_idx == sampling_params.n_sampled_values); + + // Pushing the sampled vector samples into the rowgroup samples + for (auto &value : current_vector_sample) { + analyze_state.rowgroup_sample.push_back(value); + } + + analyze_state.vectors_sampled_count++; + return true; +} + +/* + * Estimate the compression size of ALPRD using the taken samples + */ +template +idx_t AlpRDFinalAnalyze(AnalyzeState &state) { + auto &analyze_state = (AlpRDAnalyzeState &)state; + double factor_of_sampling = 1 / ((double)analyze_state.rowgroup_sample.size() / analyze_state.total_values_count); + + // Finding which is the best dictionary for the sample + double estimated_bits_per_value = + alp::AlpRDCompression::FindBestDictionary(analyze_state.rowgroup_sample, analyze_state.state); + double estimated_compressed_bits = estimated_bits_per_value * analyze_state.rowgroup_sample.size(); + double estimed_compressed_bytes = estimated_compressed_bits / 8; + + //! Overhead per segment: [Pointer to metadata + right bitwidth + left bitwidth + n dict elems] + Dictionary Size + double per_segment_overhead = AlpRDConstants::HEADER_SIZE + AlpRDConstants::MAX_DICTIONARY_SIZE_BYTES; + + //! Overhead per vector: Pointer to data + Exceptions count + double per_vector_overhead = AlpRDConstants::METADATA_POINTER_SIZE + AlpRDConstants::EXCEPTIONS_COUNT_SIZE; + + uint32_t n_vectors = std::ceil((double)analyze_state.total_values_count / AlpRDConstants::ALP_VECTOR_SIZE); + + auto estimated_size = (estimed_compressed_bytes * factor_of_sampling) + (n_vectors * per_vector_overhead); + uint32_t estimated_n_blocks = std::ceil(estimated_size / (Storage::BLOCK_SIZE - per_segment_overhead)); + + auto final_analyze_size = estimated_size + (estimated_n_blocks * per_segment_overhead); + return final_analyze_size; +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_compress.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_compress.hpp new file mode 100644 index 000000000..23e74845f --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_compress.hpp @@ -0,0 +1,301 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alprd/alprd_compress.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/patas/patas.hpp" +#include "duckdb/storage/compression/alprd/algorithm/alprd.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/storage/compression/alprd/alprd_analyze.hpp" +#include "duckdb/storage/compression/alprd/alprd_constants.hpp" +#include "duckdb/storage/compression/alp/alp_constants.hpp" + +#include "duckdb/common/helper.hpp" +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" + +#include + +namespace duckdb { + +template +struct AlpRDCompressionState : public CompressionState { + +public: + using EXACT_TYPE = typename FloatingToExact::type; + explicit AlpRDCompressionState(ColumnDataCheckpointer &checkpointer, AlpRDAnalyzeState *analyze_state) + : checkpointer(checkpointer), + function(checkpointer.GetCompressionFunction(CompressionType::COMPRESSION_ALPRD)) { + //! State variables from the analyze step that are needed for compression + state.left_parts_dict_map = std::move(analyze_state->state.left_parts_dict_map); + state.left_bit_width = analyze_state->state.left_bit_width; + state.right_bit_width = analyze_state->state.right_bit_width; + state.actual_dictionary_size = analyze_state->state.actual_dictionary_size; + actual_dictionary_size_bytes = state.actual_dictionary_size * AlpRDConstants::DICTIONARY_ELEMENT_SIZE; + next_vector_byte_index_start = AlpRDConstants::HEADER_SIZE + actual_dictionary_size_bytes; + memcpy((void *)state.left_parts_dict, (void *)analyze_state->state.left_parts_dict, + actual_dictionary_size_bytes); + CreateEmptySegment(checkpointer.GetRowGroup().start); + } + + ColumnDataCheckpointer &checkpointer; + CompressionFunction &function; + unique_ptr current_segment; + BufferHandle handle; + + idx_t vector_idx = 0; + idx_t nulls_idx = 0; + idx_t vectors_flushed = 0; + idx_t data_bytes_used = 0; + + data_ptr_t data_ptr; // Pointer to next free spot in segment; + data_ptr_t metadata_ptr; // Reverse pointer to the next free spot for the metadata; used in decoding to SKIP vectors + uint32_t actual_dictionary_size_bytes; + uint32_t next_vector_byte_index_start; + + EXACT_TYPE input_vector[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t vector_null_positions[AlpRDConstants::ALP_VECTOR_SIZE]; + + alp::AlpRDCompressionState state; + +public: + // Returns the space currently used in the segment (in bytes) + idx_t UsedSpace() const { + //! [Pointer to metadata + right bitwidth] + Dictionary Size + Bytes already used in the segment + return AlpRDConstants::HEADER_SIZE + actual_dictionary_size_bytes + data_bytes_used; + } + + // Returns the required space to store the newly compressed vector + idx_t RequiredSpace() { + idx_t required_space = + state.left_bit_packed_size + state.right_bit_packed_size + + state.exceptions_count * (AlpRDConstants::EXCEPTION_SIZE + AlpRDConstants::EXCEPTION_POSITION_SIZE) + + AlpRDConstants::EXCEPTIONS_COUNT_SIZE; + return required_space; + } + + bool HasEnoughSpace() { + //! If [start of block + used space + required space] is more than whats left (current position + //! of metadata pointer - the size of a new metadata pointer) + if ((handle.Ptr() + AlignValue(UsedSpace() + RequiredSpace())) >= + (metadata_ptr - AlpRDConstants::METADATA_POINTER_SIZE)) { + return false; + } + return true; + } + + void ResetVector() { + state.Reset(); + } + + void CreateEmptySegment(idx_t row_start) { + auto &db = checkpointer.GetDatabase(); + auto &type = checkpointer.GetType(); + auto compressed_segment = ColumnSegment::CreateTransientSegment(db, type, row_start); + compressed_segment->function = function; + current_segment = std::move(compressed_segment); + + auto &buffer_manager = BufferManager::GetBufferManager(db); + handle = buffer_manager.Pin(current_segment->block); + + // Pointer to the start of the compressed data + data_ptr = handle.Ptr() + current_segment->GetBlockOffset() + AlpRDConstants::HEADER_SIZE + + actual_dictionary_size_bytes; + // Pointer to the start of the Metadata + metadata_ptr = handle.Ptr() + current_segment->GetBlockOffset() + Storage::BLOCK_SIZE; + + next_vector_byte_index_start = AlpRDConstants::HEADER_SIZE + actual_dictionary_size_bytes; + } + + void CompressVector() { + if (nulls_idx) { + alp::AlpUtils::FindAndReplaceNullsInVector(input_vector, vector_null_positions, vector_idx, + nulls_idx); + } + alp::AlpRDCompression::Compress(input_vector, vector_idx, state); + //! Check if the compressed vector fits on current segment + if (!HasEnoughSpace()) { + auto row_start = current_segment->start + current_segment->count; + FlushSegment(); + CreateEmptySegment(row_start); + } + if (vector_idx != nulls_idx) { //! At least there is one valid value in the vector + for (idx_t i = 0; i < vector_idx; i++) { + T floating_point_value = Load(const_data_ptr_cast(&input_vector[i])); + NumericStats::Update(current_segment->stats.statistics, floating_point_value); + } + } + current_segment->count += vector_idx; + FlushVector(); + } + + // Stores the vector and its metadata + void FlushVector() { + Store(state.exceptions_count, data_ptr); + data_ptr += AlpRDConstants::EXCEPTIONS_COUNT_SIZE; + + memcpy((void *)data_ptr, (void *)state.left_parts_encoded, state.left_bit_packed_size); + data_ptr += state.left_bit_packed_size; + + memcpy((void *)data_ptr, (void *)state.right_parts_encoded, state.right_bit_packed_size); + data_ptr += state.right_bit_packed_size; + + if (state.exceptions_count > 0) { + memcpy((void *)data_ptr, (void *)state.exceptions, AlpRDConstants::EXCEPTION_SIZE * state.exceptions_count); + data_ptr += AlpRDConstants::EXCEPTION_SIZE * state.exceptions_count; + memcpy((void *)data_ptr, (void *)state.exceptions_positions, + AlpRDConstants::EXCEPTION_POSITION_SIZE * state.exceptions_count); + data_ptr += AlpRDConstants::EXCEPTION_POSITION_SIZE * state.exceptions_count; + } + + data_bytes_used += + state.left_bit_packed_size + state.right_bit_packed_size + + (state.exceptions_count * (AlpRDConstants::EXCEPTION_SIZE + AlpRDConstants::EXCEPTION_POSITION_SIZE)) + + AlpRDConstants::EXCEPTIONS_COUNT_SIZE; + + // Write pointer to the vector data (metadata) + metadata_ptr -= AlpRDConstants::METADATA_POINTER_SIZE; + Store(next_vector_byte_index_start, metadata_ptr); + next_vector_byte_index_start = UsedSpace(); + + vectors_flushed++; + vector_idx = 0; + nulls_idx = 0; + ResetVector(); + } + + void FlushSegment() { + auto &checkpoint_state = checkpointer.GetCheckpointState(); + auto dataptr = handle.Ptr(); + + idx_t metadata_offset = AlignValue(UsedSpace()); + + // Verify that the metadata_ptr is not smaller than the space used by the data + D_ASSERT(dataptr + metadata_offset <= metadata_ptr); + + idx_t bytes_used_by_metadata = dataptr + Storage::BLOCK_SIZE - metadata_ptr; + + // Initially the total segment size is the size of the block + idx_t total_segment_size = Storage::BLOCK_SIZE; + + //! We compact the block if the space used is less than a threshold + const auto used_space_percentage = + static_cast(metadata_offset + bytes_used_by_metadata) / static_cast(total_segment_size); + if (used_space_percentage < AlpConstants::COMPACT_BLOCK_THRESHOLD) { +#ifdef DEBUG + //! Copy the first 4 bytes of the metadata + uint32_t verify_bytes; + memcpy((void *)&verify_bytes, metadata_ptr, 4); +#endif + memmove(dataptr + metadata_offset, metadata_ptr, bytes_used_by_metadata); +#ifdef DEBUG + //! Now assert that the memmove was correct + D_ASSERT(verify_bytes == *(uint32_t *)(dataptr + metadata_offset)); +#endif + total_segment_size = metadata_offset + bytes_used_by_metadata; + } + + // Store the offset to the end of metadata (to be used as a backwards pointer in decoding) + Store(total_segment_size, dataptr); + dataptr += AlpRDConstants::METADATA_POINTER_SIZE; + + // Store the right bw for the segment + Store(state.right_bit_width, dataptr); + dataptr += AlpRDConstants::RIGHT_BIT_WIDTH_SIZE; + + // Store the left bw for the segment + Store(state.left_bit_width, dataptr); + dataptr += AlpRDConstants::LEFT_BIT_WIDTH_SIZE; + + // Store the actual number of elements on the dictionary of the segment + Store(state.actual_dictionary_size, dataptr); + dataptr += AlpRDConstants::N_DICTIONARY_ELEMENTS_SIZE; + + // Store the Dictionary + memcpy((void *)dataptr, (void *)state.left_parts_dict, actual_dictionary_size_bytes); + + handle.Destroy(); + checkpoint_state.FlushSegment(std::move(current_segment), total_segment_size); + data_bytes_used = 0; + vectors_flushed = 0; + } + + void Finalize() { + if (vector_idx != 0) { + CompressVector(); + } + FlushSegment(); + current_segment.reset(); + } + + void Append(UnifiedVectorFormat &vdata, idx_t count) { + auto data = UnifiedVectorFormat::GetData(vdata); + idx_t values_left_in_data = count; + idx_t offset_in_data = 0; + while (values_left_in_data > 0) { + // We calculate until which value in data we must go to fill the input_vector + // to avoid checking if input_vector is filled in each iteration + auto values_to_fill_alp_input = + MinValue(AlpConstants::ALP_VECTOR_SIZE - vector_idx, values_left_in_data); + if (vdata.validity.AllValid()) { //! We optimize a loop when there are no null + for (idx_t i = 0; i < values_to_fill_alp_input; i++) { + auto idx = vdata.sel->get_index(offset_in_data + i); + EXACT_TYPE value = Load(const_data_ptr_cast(&data[idx])); + input_vector[vector_idx + i] = value; + } + } else { + for (idx_t i = 0; i < values_to_fill_alp_input; i++) { + auto idx = vdata.sel->get_index(offset_in_data + i); + EXACT_TYPE value = Load(const_data_ptr_cast(&data[idx])); + bool is_null = !vdata.validity.RowIsValid(idx); + //! We resolve null values with a predicated comparison + vector_null_positions[nulls_idx] = vector_idx + i; + nulls_idx += is_null; + input_vector[vector_idx + i] = value; + } + } + offset_in_data += values_to_fill_alp_input; + values_left_in_data -= values_to_fill_alp_input; + vector_idx += values_to_fill_alp_input; + // We still need this check since we could have an incomplete input_vector at the end of data + if (vector_idx == AlpConstants::ALP_VECTOR_SIZE) { + CompressVector(); + D_ASSERT(vector_idx == 0); + } + } + } +}; + +template +unique_ptr AlpRDInitCompression(ColumnDataCheckpointer &checkpointer, + unique_ptr state) { + return make_uniq>(checkpointer, (AlpRDAnalyzeState *)state.get()); +} + +template +void AlpRDCompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { + auto &state = (AlpRDCompressionState &)state_p; + UnifiedVectorFormat vdata; + scan_vector.ToUnifiedFormat(count, vdata); + state.Append(vdata, count); +} + +template +void AlpRDFinalizeCompress(CompressionState &state_p) { + auto &state = (AlpRDCompressionState &)state_p; + state.Finalize(); +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_constants.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_constants.hpp new file mode 100644 index 000000000..521551e2d --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_constants.hpp @@ -0,0 +1,35 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alprd/alprd_constants.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +namespace duckdb { + +class AlpRDConstants { +public: + static constexpr uint32_t ALP_VECTOR_SIZE = 1024; + + static constexpr uint8_t MAX_DICTIONARY_BIT_WIDTH = 3; + static constexpr uint8_t MAX_DICTIONARY_SIZE = (1 << MAX_DICTIONARY_BIT_WIDTH); // 8 + static constexpr uint8_t CUTTING_LIMIT = 16; + static constexpr uint8_t DICTIONARY_ELEMENT_SIZE = sizeof(uint16_t); + static constexpr uint8_t MAX_DICTIONARY_SIZE_BYTES = MAX_DICTIONARY_SIZE * DICTIONARY_ELEMENT_SIZE; + + static constexpr uint8_t EXCEPTION_SIZE = sizeof(uint16_t); + static constexpr uint8_t METADATA_POINTER_SIZE = sizeof(uint32_t); + static constexpr uint8_t EXCEPTIONS_COUNT_SIZE = sizeof(uint16_t); + static constexpr uint8_t EXCEPTION_POSITION_SIZE = sizeof(uint16_t); + static constexpr uint8_t RIGHT_BIT_WIDTH_SIZE = sizeof(uint8_t); + static constexpr uint8_t LEFT_BIT_WIDTH_SIZE = sizeof(uint8_t); + static constexpr uint8_t N_DICTIONARY_ELEMENTS_SIZE = sizeof(uint8_t); + static constexpr uint8_t HEADER_SIZE = + METADATA_POINTER_SIZE + RIGHT_BIT_WIDTH_SIZE + LEFT_BIT_WIDTH_SIZE + + N_DICTIONARY_ELEMENTS_SIZE; // Pointer to metadata + Right BW + Left BW + Dict Elems +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_fetch.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_fetch.hpp new file mode 100644 index 000000000..c4b36c2fa --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_fetch.hpp @@ -0,0 +1,41 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alp/alp_fetch.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/alprd/alprd_scan.hpp" + +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" + +namespace duckdb { + +template +void AlpRDFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t row_id, Vector &result, idx_t result_idx) { + using EXACT_TYPE = typename FloatingToExact::type; + AlpRDScanState scan_state(segment); + scan_state.Skip(segment, row_id); + auto result_data = FlatVector::GetData(result); + result_data[result_idx] = (EXACT_TYPE)0; + + if (scan_state.VectorFinished() && scan_state.total_value_count < scan_state.count) { + scan_state.LoadVector(scan_state.vector_state.decoded_values); + } + scan_state.vector_state.Scan((uint8_t *)(result_data + result_idx), 1); + scan_state.total_value_count++; +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_scan.hpp b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_scan.hpp new file mode 100644 index 000000000..efaddbd2e --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/compression/alprd/alprd_scan.hpp @@ -0,0 +1,252 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/compression/alprd/alprd_scan.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/storage/compression/alprd/algorithm/alprd.hpp" +#include "duckdb/storage/compression/alprd/alprd_constants.hpp" + +#include "duckdb/common/limits.hpp" +#include "duckdb/common/types/null_value.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/main/config.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +#include "duckdb/storage/table/column_data_checkpointer.hpp" +#include "duckdb/storage/table/column_segment.hpp" +#include "duckdb/common/operator/subtract.hpp" +#include "duckdb/storage/table/scan_state.hpp" + +namespace duckdb { + +template +struct AlpRDVectorState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + void Reset() { + index = 0; + } + + // Scan of the data itself + template + void Scan(uint8_t *dest, idx_t count) { + if (!SKIP) { + memcpy(dest, (void *)(decoded_values + index), sizeof(T) * count); + } + index += count; + } + + template + void LoadValues(EXACT_TYPE *values_buffer, idx_t count) { + if (SKIP) { + return; + } + values_buffer[0] = (EXACT_TYPE)0; + alp::AlpRDDecompression::Decompress(left_encoded, right_encoded, left_parts_dict, values_buffer, count, + exceptions_count, exceptions, exceptions_positions, left_bit_width, + right_bit_width); + } + +public: + idx_t index; + uint8_t left_encoded[AlpRDConstants::ALP_VECTOR_SIZE * 8]; + uint8_t right_encoded[AlpRDConstants::ALP_VECTOR_SIZE * 8]; + EXACT_TYPE decoded_values[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions_positions[AlpRDConstants::ALP_VECTOR_SIZE]; + uint16_t exceptions_count; + uint8_t right_bit_width; + uint8_t left_bit_width; + uint16_t left_parts_dict[AlpRDConstants::MAX_DICTIONARY_SIZE]; +}; + +template +struct AlpRDScanState : public SegmentScanState { +public: + using EXACT_TYPE = typename FloatingToExact::type; + + explicit AlpRDScanState(ColumnSegment &segment) : segment(segment), count(segment.count) { + auto &buffer_manager = BufferManager::GetBufferManager(segment.db); + + handle = buffer_manager.Pin(segment.block); + // ScanStates never exceed the boundaries of a Segment, + // but are not guaranteed to start at the beginning of the Block + segment_data = handle.Ptr() + segment.GetBlockOffset(); + auto metadata_offset = Load(segment_data); + metadata_ptr = segment_data + metadata_offset; + + // Load the Right Bit Width which is in the segment header after the pointer to the first metadata + vector_state.right_bit_width = Load(segment_data + AlpRDConstants::METADATA_POINTER_SIZE); + vector_state.left_bit_width = + Load(segment_data + AlpRDConstants::METADATA_POINTER_SIZE + AlpRDConstants::RIGHT_BIT_WIDTH_SIZE); + + uint8_t actual_dictionary_size = + Load(segment_data + AlpRDConstants::METADATA_POINTER_SIZE + AlpRDConstants::RIGHT_BIT_WIDTH_SIZE + + AlpRDConstants::LEFT_BIT_WIDTH_SIZE); + uint8_t actual_dictionary_size_bytes = actual_dictionary_size * AlpRDConstants::DICTIONARY_ELEMENT_SIZE; + + // Load the left parts dictionary which is after the segment header and is of a fixed size + memcpy(vector_state.left_parts_dict, (void *)(segment_data + AlpRDConstants::HEADER_SIZE), + actual_dictionary_size_bytes); + } + + BufferHandle handle; + data_ptr_t metadata_ptr; + data_ptr_t segment_data; + idx_t total_value_count = 0; + AlpRDVectorState vector_state; + + ColumnSegment &segment; + idx_t count; + + idx_t LeftInVector() const { + return AlpRDConstants::ALP_VECTOR_SIZE - (total_value_count % AlpRDConstants::ALP_VECTOR_SIZE); + } + + inline bool VectorFinished() const { + return (total_value_count % AlpRDConstants::ALP_VECTOR_SIZE) == 0; + } + + // Scan up to a vector boundary + template + void ScanVector(EXACT_TYPE *values, idx_t vector_size) { + D_ASSERT(vector_size <= AlpRDConstants::ALP_VECTOR_SIZE); + D_ASSERT(vector_size <= LeftInVector()); + if (VectorFinished() && total_value_count < count) { + if (vector_size == AlpRDConstants::ALP_VECTOR_SIZE) { + LoadVector(values); + total_value_count += vector_size; + return; + } else { + // Even if SKIP is given, the vector size is not big enough to be able to fully skip the entire vector + LoadVector(vector_state.decoded_values); + } + } + vector_state.template Scan((uint8_t *)values, vector_size); + + total_value_count += vector_size; + } + + // Using the metadata, we can avoid loading any of the data if we don't care about the vector at all + void SkipVector() { + // Skip the offset indicating where the data starts + metadata_ptr -= AlpRDConstants::METADATA_POINTER_SIZE; + idx_t vector_size = MinValue((idx_t)AlpRDConstants::ALP_VECTOR_SIZE, count - total_value_count); + total_value_count += vector_size; + } + + template + void LoadVector(EXACT_TYPE *value_buffer) { + vector_state.Reset(); + + // Load the offset (metadata) indicating where the vector data starts + metadata_ptr -= AlpRDConstants::METADATA_POINTER_SIZE; + auto data_byte_offset = Load(metadata_ptr); + D_ASSERT(data_byte_offset < Storage::BLOCK_SIZE); + + idx_t vector_size = MinValue((idx_t)AlpRDConstants::ALP_VECTOR_SIZE, (count - total_value_count)); + + data_ptr_t vector_ptr = segment_data + data_byte_offset; + + // Load the vector data + vector_state.exceptions_count = Load(vector_ptr); + vector_ptr += AlpRDConstants::EXCEPTIONS_COUNT_SIZE; + D_ASSERT(vector_state.exceptions_count <= vector_size); + + auto left_bp_size = BitpackingPrimitives::GetRequiredSize(vector_size, vector_state.left_bit_width); + auto right_bp_size = BitpackingPrimitives::GetRequiredSize(vector_size, vector_state.right_bit_width); + + memcpy(vector_state.left_encoded, (void *)vector_ptr, left_bp_size); + vector_ptr += left_bp_size; + + memcpy(vector_state.right_encoded, (void *)vector_ptr, right_bp_size); + vector_ptr += right_bp_size; + + if (vector_state.exceptions_count > 0) { + memcpy(vector_state.exceptions, (void *)vector_ptr, + AlpRDConstants::EXCEPTION_SIZE * vector_state.exceptions_count); + vector_ptr += AlpRDConstants::EXCEPTION_SIZE * vector_state.exceptions_count; + memcpy(vector_state.exceptions_positions, (void *)vector_ptr, + AlpRDConstants::EXCEPTION_POSITION_SIZE * vector_state.exceptions_count); + } + + // Decode all the vector values to the specified 'value_buffer' + vector_state.template LoadValues(value_buffer, vector_size); + } + +public: + //! Skip the next 'skip_count' values, we don't store the values + void Skip(ColumnSegment &col_segment, idx_t skip_count) { + + if (total_value_count != 0 && !VectorFinished()) { + // Finish skipping the current vector + idx_t to_skip = LeftInVector(); + skip_count -= to_skip; + ScanVector(nullptr, to_skip); + } + // Figure out how many entire vectors we can skip + // For these vectors, we don't even need to process the metadata or values + idx_t vectors_to_skip = skip_count / AlpRDConstants::ALP_VECTOR_SIZE; + for (idx_t i = 0; i < vectors_to_skip; i++) { + SkipVector(); + } + skip_count -= AlpRDConstants::ALP_VECTOR_SIZE * vectors_to_skip; + if (skip_count == 0) { + return; + } + // For the last vector that this skip (partially) touches, we do need to + // load the metadata and values into the vector_state because + // we don't know exactly how many they are + ScanVector(nullptr, skip_count); + } +}; + +template +unique_ptr AlpRDInitScan(ColumnSegment &segment) { + auto result = make_uniq_base>(segment); + return result; +} + +//===--------------------------------------------------------------------===// +// Scan base data +//===--------------------------------------------------------------------===// +template +void AlpRDScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result, + idx_t result_offset) { + using EXACT_TYPE = typename FloatingToExact::type; + auto &scan_state = (AlpRDScanState &)*state.scan_state; + + // Get the pointer to the result values + auto current_result_ptr = FlatVector::GetData(result); + result.SetVectorType(VectorType::FLAT_VECTOR); + current_result_ptr += result_offset; + + idx_t scanned = 0; + while (scanned < scan_count) { + const auto remaining = scan_count - scanned; + const idx_t to_scan = MinValue(remaining, scan_state.LeftInVector()); + + scan_state.template ScanVector(current_result_ptr + scanned, to_scan); + scanned += to_scan; + } +} + +template +void AlpRDSkip(ColumnSegment &segment, ColumnScanState &state, idx_t skip_count) { + auto &scan_state = (AlpRDScanState &)*state.scan_state; + scan_state.Skip(segment, skip_count); +} + +template +void AlpRDScan(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result) { + AlpRDScanPartial(segment, state, scan_count, result, 0); +} + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_analyze.hpp b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_analyze.hpp index 1c2fc3300..2b56d15ba 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_analyze.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_analyze.hpp @@ -16,120 +16,24 @@ namespace duckdb { struct EmptyChimpWriter; template -struct ChimpAnalyzeState : public AnalyzeState { -public: - using CHIMP_TYPE = typename ChimpType::type; - - ChimpAnalyzeState() : state() { - state.AssignDataBuffer(nullptr); - } - ChimpState state; - idx_t group_idx = 0; - idx_t data_byte_size = 0; - idx_t metadata_byte_size = 0; - -public: - void WriteValue(CHIMP_TYPE value, bool is_valid) { - if (!is_valid) { - return; - } - //! Keep track of when a segment would end, to accurately simulate Reset()s in compress step - if (!HasEnoughSpace()) { - StartNewSegment(); - } - Chimp128Compression::Store(value, state.chimp); - group_idx++; - if (group_idx == ChimpPrimitives::CHIMP_SEQUENCE_SIZE) { - StartNewGroup(); - } - } - - void StartNewSegment() { - state.Flush(); - StartNewGroup(); - data_byte_size += UsedSpace(); - metadata_byte_size += ChimpPrimitives::HEADER_SIZE; - state.chimp.output.SetStream(nullptr); - } - - idx_t CurrentGroupMetadataSize() const { - idx_t metadata_size = 0; - - metadata_size += 3 * state.chimp.leading_zero_buffer.BlockCount(); - metadata_size += state.chimp.flag_buffer.BytesUsed(); - metadata_size += 2 * state.chimp.packed_data_buffer.index; - return metadata_size; - } - - idx_t RequiredSpace() const { - idx_t required_space = ChimpPrimitives::MAX_BYTES_PER_VALUE; - // Any value could be the last, - // so the cost of flushing metadata should be factored into the cost - // byte offset of data - required_space += sizeof(byte_index_t); - // amount of leading zero blocks - required_space += sizeof(uint8_t); - // first leading zero block - required_space += 3; - // amount of flag bytes - required_space += sizeof(uint8_t); - // first flag byte - required_space += 1; - return required_space; - } - - void StartNewGroup() { - metadata_byte_size += CurrentGroupMetadataSize(); - group_idx = 0; - state.chimp.Reset(); - } - - idx_t UsedSpace() const { - return state.chimp.output.BytesWritten(); - } - - bool HasEnoughSpace() { - idx_t total_bytes_used = 0; - total_bytes_used += AlignValue(ChimpPrimitives::HEADER_SIZE + UsedSpace() + RequiredSpace()); - total_bytes_used += CurrentGroupMetadataSize(); - total_bytes_used += metadata_byte_size; - return total_bytes_used <= Storage::BLOCK_SIZE; - } - - idx_t TotalUsedBytes() const { - return metadata_byte_size + AlignValue(data_byte_size + UsedSpace()); - } -}; +struct ChimpAnalyzeState : public AnalyzeState {}; template unique_ptr ChimpInitAnalyze(ColumnData &col_data, PhysicalType type) { - return make_uniq>(); + // This compression type is deprecated + return nullptr; } template bool ChimpAnalyze(AnalyzeState &state, Vector &input, idx_t count) { - using CHIMP_TYPE = typename ChimpType::type; - auto &analyze_state = (ChimpAnalyzeState &)state; - UnifiedVectorFormat vdata; - input.ToUnifiedFormat(count, vdata); - - auto data = UnifiedVectorFormat::GetData(vdata); - for (idx_t i = 0; i < count; i++) { - auto idx = vdata.sel->get_index(i); - analyze_state.WriteValue(data[idx], vdata.validity.RowIsValid(idx)); - } - return true; + throw InternalException("Chimp has been deprecated, can no longer be used to compress data"); + return false; } template idx_t ChimpFinalAnalyze(AnalyzeState &state) { - auto &chimp = (ChimpAnalyzeState &)state; - // Finish the last "segment" - chimp.StartNewSegment(); - // Multiply the final size to factor in the extra cost of decompression time - const auto multiplier = 2.0; - const auto final_analyze_size = chimp.TotalUsedBytes(); - return final_analyze_size * multiplier; + throw InternalException("Chimp has been deprecated, can no longer be used to compress data"); + return DConstants::INVALID_INDEX; } } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_compress.hpp b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_compress.hpp index a514505b6..a233246f3 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_compress.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_compress.hpp @@ -28,254 +28,25 @@ namespace duckdb { template -struct ChimpCompressionState : public CompressionState { -public: - using CHIMP_TYPE = typename ChimpType::type; - - explicit ChimpCompressionState(ColumnDataCheckpointer &checkpointer, ChimpAnalyzeState *analyze_state) - : checkpointer(checkpointer), - function(checkpointer.GetCompressionFunction(CompressionType::COMPRESSION_CHIMP)) { - CreateEmptySegment(checkpointer.GetRowGroup().start); - - // These buffers are recycled for every group, so they only have to be set once - state.AssignLeadingZeroBuffer((uint8_t *)leading_zero_blocks); - state.AssignFlagBuffer((uint8_t *)flags); - state.AssignPackedDataBuffer((uint16_t *)packed_data_blocks); - } - - ColumnDataCheckpointer &checkpointer; - CompressionFunction &function; - unique_ptr current_segment; - BufferHandle handle; - idx_t group_idx = 0; - uint8_t flags[ChimpPrimitives::CHIMP_SEQUENCE_SIZE / 4]; - uint8_t leading_zero_blocks[ChimpPrimitives::LEADING_ZERO_BLOCK_BUFFERSIZE]; - uint16_t packed_data_blocks[ChimpPrimitives::CHIMP_SEQUENCE_SIZE]; - - // Ptr to next free spot in segment; - data_ptr_t segment_data; - data_ptr_t metadata_ptr; - uint32_t next_group_byte_index_start = ChimpPrimitives::HEADER_SIZE; - // The total size of metadata in the current segment - idx_t metadata_byte_size = 0; - - ChimpState state; - -public: - idx_t RequiredSpace() const { - idx_t required_space = ChimpPrimitives::MAX_BYTES_PER_VALUE; - // Any value could be the last, - // so the cost of flushing metadata should be factored into the cost - - // byte offset of data - required_space += sizeof(byte_index_t); - // amount of leading zero blocks - required_space += sizeof(uint8_t); - // first leading zero block - required_space += 3; - // amount of flag bytes - required_space += sizeof(uint8_t); - // first flag byte - required_space += 1; - return required_space; - } - - // How many bytes the data occupies for the current segment - idx_t UsedSpace() const { - return state.chimp.output.BytesWritten(); - } - - idx_t RemainingSpace() const { - return metadata_ptr - (handle.Ptr() + UsedSpace()); - } - - idx_t CurrentGroupMetadataSize() const { - idx_t metadata_size = 0; - - metadata_size += 3 * state.chimp.leading_zero_buffer.BlockCount(); - metadata_size += state.chimp.flag_buffer.BytesUsed(); - metadata_size += 2 * state.chimp.packed_data_buffer.index; - return metadata_size; - } - - // The current segment has enough space to fit this new value - bool HasEnoughSpace() { - if (handle.Ptr() + AlignValue(ChimpPrimitives::HEADER_SIZE + UsedSpace() + RequiredSpace()) >= - (metadata_ptr - CurrentGroupMetadataSize())) { - return false; - } - return true; - } - - void CreateEmptySegment(idx_t row_start) { - group_idx = 0; - metadata_byte_size = 0; - auto &db = checkpointer.GetDatabase(); - auto &type = checkpointer.GetType(); - auto compressed_segment = ColumnSegment::CreateTransientSegment(db, type, row_start); - compressed_segment->function = function; - current_segment = std::move(compressed_segment); - next_group_byte_index_start = ChimpPrimitives::HEADER_SIZE; - - auto &buffer_manager = BufferManager::GetBufferManager(db); - handle = buffer_manager.Pin(current_segment->block); - - segment_data = handle.Ptr() + current_segment->GetBlockOffset() + ChimpPrimitives::HEADER_SIZE; - metadata_ptr = handle.Ptr() + current_segment->GetBlockOffset() + Storage::BLOCK_SIZE; - state.AssignDataBuffer(segment_data); - state.chimp.Reset(); - } - - void Append(UnifiedVectorFormat &vdata, idx_t count) { - auto data = UnifiedVectorFormat::GetData(vdata); - - for (idx_t i = 0; i < count; i++) { - auto idx = vdata.sel->get_index(i); - WriteValue(data[idx], vdata.validity.RowIsValid(idx)); - } - } - - void WriteValue(CHIMP_TYPE value, bool is_valid) { - if (!HasEnoughSpace()) { - // Segment is full - auto row_start = current_segment->start + current_segment->count; - FlushSegment(); - CreateEmptySegment(row_start); - } - current_segment->count++; - - if (is_valid) { - T floating_point_value = Load(const_data_ptr_cast(&value)); - NumericStats::Update(current_segment->stats.statistics, floating_point_value); - } else { - //! FIXME: find a cheaper alternative to storing a NULL - // store this as "value_identical", only using 9 bits for a NULL - value = state.chimp.previous_value; - } - - Chimp128Compression::Store(value, state.chimp); - group_idx++; - if (group_idx == ChimpPrimitives::CHIMP_SEQUENCE_SIZE) { - FlushGroup(); - } - } - - void FlushGroup() { - // Has to be called first to flush the last values in the LeadingZeroBuffer - state.chimp.Flush(); - - metadata_ptr -= sizeof(byte_index_t); - metadata_byte_size += sizeof(byte_index_t); - // Store where this groups data starts, relative to the start of the segment - Store(next_group_byte_index_start, metadata_ptr); - next_group_byte_index_start = UsedSpace(); - - const uint8_t leading_zero_block_count = state.chimp.leading_zero_buffer.BlockCount(); - // Every 8 values are packed in one block - D_ASSERT(leading_zero_block_count <= ChimpPrimitives::CHIMP_SEQUENCE_SIZE / 8); - metadata_ptr -= sizeof(uint8_t); - metadata_byte_size += sizeof(uint8_t); - // Store how many leading zero blocks there are - Store(leading_zero_block_count, metadata_ptr); - - const uint64_t bytes_used_by_leading_zero_blocks = 3 * leading_zero_block_count; - metadata_ptr -= bytes_used_by_leading_zero_blocks; - metadata_byte_size += bytes_used_by_leading_zero_blocks; - // Store the leading zeros (8 per 3 bytes) for this group - memcpy((void *)metadata_ptr, (void *)leading_zero_blocks, bytes_used_by_leading_zero_blocks); - - //! This is max 1024, because it's the amount of flags there are, not the amount of bytes that takes up - const uint16_t flag_bytes = state.chimp.flag_buffer.BytesUsed(); -#ifdef DEBUG - const idx_t padding = (current_segment->count % ChimpPrimitives::CHIMP_SEQUENCE_SIZE) == 0 - ? ChimpPrimitives::CHIMP_SEQUENCE_SIZE - : 0; - const idx_t size_of_group = padding + current_segment->count % ChimpPrimitives::CHIMP_SEQUENCE_SIZE; - D_ASSERT((AlignValue(size_of_group - 1) / 4) == flag_bytes); -#endif - - metadata_ptr -= flag_bytes; - metadata_byte_size += flag_bytes; - // Store the flags (4 per byte) for this group - memcpy((void *)metadata_ptr, (void *)flags, flag_bytes); - - // Store the packed data blocks (2 bytes each) - // We dont need to store an extra count for this, - // as the count can be derived from unpacking the flags and counting the '1' flags - - // FIXME: this does stop us from skipping groups with point queries, - // because the metadata has a variable size, and we have to extract all flags + iterate them to know this size - const uint16_t packed_data_blocks_count = state.chimp.packed_data_buffer.index; - metadata_ptr -= packed_data_blocks_count * 2; - metadata_byte_size += packed_data_blocks_count * 2; - if ((uint64_t)metadata_ptr & 1) { - // Align on a two-byte boundary - metadata_ptr--; - metadata_byte_size++; - } - memcpy((void *)metadata_ptr, (void *)packed_data_blocks, packed_data_blocks_count * sizeof(uint16_t)); - - state.chimp.Reset(); - group_idx = 0; - } - - // FIXME: only do this if the wasted space meets a certain threshold (>= 20%) - void FlushSegment() { - if (group_idx) { - // Only call this when the group actually has data that needs to be flushed - FlushGroup(); - } - state.chimp.output.Flush(); - auto &checkpoint_state = checkpointer.GetCheckpointState(); - auto dataptr = handle.Ptr(); - - // Compact the segment by moving the metadata next to the data. - idx_t bytes_used_by_data = ChimpPrimitives::HEADER_SIZE + UsedSpace(); - idx_t metadata_offset = AlignValue(bytes_used_by_data); - // Verify that the metadata_ptr does not cross this threshold - D_ASSERT(dataptr + metadata_offset <= metadata_ptr); - idx_t metadata_size = dataptr + Storage::BLOCK_SIZE - metadata_ptr; - idx_t total_segment_size = metadata_offset + metadata_size; -#ifdef DEBUG - uint32_t verify_bytes; - memcpy((void *)&verify_bytes, metadata_ptr, 4); -#endif - memmove(dataptr + metadata_offset, metadata_ptr, metadata_size); -#ifdef DEBUG - D_ASSERT(verify_bytes == *(uint32_t *)(dataptr + metadata_offset)); -#endif - // Store the offset of the metadata of the first group (which is at the highest address). - Store(metadata_offset + metadata_size, dataptr); - handle.Destroy(); - checkpoint_state.FlushSegment(std::move(current_segment), total_segment_size); - } - - void Finalize() { - FlushSegment(); - current_segment.reset(); - } -}; +struct ChimpCompressionState : public CompressionState {}; // Compression Functions template unique_ptr ChimpInitCompression(ColumnDataCheckpointer &checkpointer, unique_ptr state) { - return make_uniq>(checkpointer, (ChimpAnalyzeState *)state.get()); + throw InternalException("Chimp has been deprecated, can no longer be used to compress data"); + return nullptr; } template void ChimpCompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { - auto &state = (ChimpCompressionState &)state_p; - UnifiedVectorFormat vdata; - scan_vector.ToUnifiedFormat(count, vdata); - state.Append(vdata, count); + throw InternalException("Chimp has been deprecated, can no longer be used to compress data"); } template void ChimpFinalizeCompress(CompressionState &state_p) { - auto &state = (ChimpCompressionState &)state_p; - state.Finalize(); + throw InternalException("Chimp has been deprecated, can no longer be used to compress data"); } } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_scan.hpp b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_scan.hpp index 559b5832d..206806a59 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_scan.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/chimp/chimp_scan.hpp @@ -263,7 +263,7 @@ template void ChimpScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result, idx_t result_offset) { using INTERNAL_TYPE = typename ChimpType::type; - auto &scan_state = (ChimpScanState &)*state.scan_state; + auto &scan_state = state.scan_state->Cast>(); T *result_data = FlatVector::GetData(result); result.SetVectorType(VectorType::FLAT_VECTOR); @@ -280,7 +280,7 @@ void ChimpScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t scan template void ChimpSkip(ColumnSegment &segment, ColumnScanState &state, idx_t skip_count) { - auto &scan_state = (ChimpScanState &)*state.scan_state; + auto &scan_state = state.scan_state->Cast>(); scan_state.Skip(segment, skip_count); } diff --git a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_analyze.hpp b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_analyze.hpp index 6a6bca991..0f9dab421 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_analyze.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_analyze.hpp @@ -16,124 +16,24 @@ namespace duckdb { struct EmptyPatasWriter; template -struct PatasAnalyzeState : public AnalyzeState { -public: - using EXACT_TYPE = typename FloatingToExact::type; - - PatasAnalyzeState() : state((void *)this) { - state.AssignDataBuffer(nullptr); - } - PatasState state; - idx_t group_idx = 0; - idx_t data_byte_size = 0; - idx_t metadata_byte_size = 0; - //! To optimally store NULL, we keep track of the directly previous value - EXACT_TYPE previous_value; - -public: - void WriteValue(EXACT_TYPE value, bool is_valid) { - if (!is_valid) { - value = previous_value; - } - //! Keep track of when a segment would end, to accurately simulate Reset()s in compress step - if (!HasEnoughSpace()) { - StartNewSegment(); - } - patas::PatasCompression::Store(value, state.patas_state); - previous_value = value; - group_idx++; - if (group_idx == PatasPrimitives::PATAS_GROUP_SIZE) { - StartNewGroup(); - } - } - - idx_t CurrentGroupMetadataSize() const { - idx_t metadata_size = 0; - - // Offset to the data of the group - metadata_size += sizeof(uint32_t); - // Packed Trailing zeros + significant bytes + index_offsets for group - metadata_size += 2 * group_idx; - return metadata_size; - } - - void StartNewSegment() { - StartNewGroup(); - data_byte_size += UsedSpace(); - metadata_byte_size += PatasPrimitives::HEADER_SIZE; - state.patas_state.byte_writer.SetStream(nullptr); - } - - idx_t RequiredSpace() const { - idx_t required_space = 0; - required_space += sizeof(EXACT_TYPE); - required_space += sizeof(uint16_t); - return required_space; - } - - void StartNewGroup() { - previous_value = 0; - metadata_byte_size += CurrentGroupMetadataSize(); - group_idx = 0; - state.patas_state.Reset(); - } - - idx_t UsedSpace() const { - return state.patas_state.byte_writer.BytesWritten(); - } - - bool HasEnoughSpace() { - idx_t total_bytes_used = 0; - total_bytes_used += AlignValue(PatasPrimitives::HEADER_SIZE + UsedSpace() + RequiredSpace()); - total_bytes_used += CurrentGroupMetadataSize(); - total_bytes_used += metadata_byte_size; - return total_bytes_used <= Storage::BLOCK_SIZE; - } - - idx_t TotalUsedBytes() const { - return metadata_byte_size + AlignValue(data_byte_size + UsedSpace()); - } -}; - -struct EmptyPatasWriter { - - template - static void Operation(VALUE_TYPE uncompressed_value, bool is_valid, void *state_p) { - using EXACT_TYPE = typename FloatingToExact::type; - - auto state_wrapper = (PatasAnalyzeState *)state_p; - state_wrapper->WriteValue(Load(const_data_ptr_cast(&uncompressed_value)), is_valid); - } -}; +struct PatasAnalyzeState : public AnalyzeState {}; template unique_ptr PatasInitAnalyze(ColumnData &col_data, PhysicalType type) { - return make_uniq>(); + // This compression type is deprecated + return nullptr; } template bool PatasAnalyze(AnalyzeState &state, Vector &input, idx_t count) { - auto &analyze_state = (PatasAnalyzeState &)state; - UnifiedVectorFormat vdata; - input.ToUnifiedFormat(count, vdata); - - auto data = UnifiedVectorFormat::GetData(vdata); - for (idx_t i = 0; i < count; i++) { - auto idx = vdata.sel->get_index(i); - analyze_state.state.template Update(data[idx], vdata.validity.RowIsValid(idx)); - } - return true; + throw InternalException("Patas has been deprecated, can no longer be used to compress data"); + return false; } template idx_t PatasFinalAnalyze(AnalyzeState &state) { - auto &patas_state = (PatasAnalyzeState &)state; - // Finish the last "segment" - patas_state.StartNewSegment(); - const auto final_analyze_size = patas_state.TotalUsedBytes(); - // Multiply the final size to factor in the extra cost of decompression time - const auto multiplier = 1.2; - return final_analyze_size * multiplier; + throw InternalException("Patas has been deprecated, can no longer be used to compress data"); + return DConstants::INVALID_INDEX; } } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_compress.hpp b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_compress.hpp index 28579dbb2..2931e9168 100644 --- a/src/duckdb/src/include/duckdb/storage/compression/patas/patas_compress.hpp +++ b/src/duckdb/src/include/duckdb/storage/compression/patas/patas_compress.hpp @@ -30,204 +30,25 @@ namespace duckdb { // State template -struct PatasCompressionState : public CompressionState { -public: - using EXACT_TYPE = typename FloatingToExact::type; - - struct PatasWriter { - - template - static void Operation(VALUE_TYPE value, bool is_valid, void *state_p) { - //! Need access to the CompressionState to be able to flush the segment - auto state_wrapper = (PatasCompressionState *)state_p; - - if (!state_wrapper->HasEnoughSpace()) { - // Segment is full - auto row_start = state_wrapper->current_segment->start + state_wrapper->current_segment->count; - state_wrapper->FlushSegment(); - state_wrapper->CreateEmptySegment(row_start); - } - - if (is_valid) { - NumericStats::Update(state_wrapper->current_segment->stats.statistics, value); - } - - state_wrapper->WriteValue(Load(const_data_ptr_cast(&value))); - } - }; - - explicit PatasCompressionState(ColumnDataCheckpointer &checkpointer, PatasAnalyzeState *analyze_state) - : checkpointer(checkpointer), - function(checkpointer.GetCompressionFunction(CompressionType::COMPRESSION_PATAS)) { - CreateEmptySegment(checkpointer.GetRowGroup().start); - - state.data_ptr = (void *)this; - state.patas_state.packed_data_buffer.SetBuffer(packed_data); - state.patas_state.Reset(); - } - - ColumnDataCheckpointer &checkpointer; - CompressionFunction &function; - unique_ptr current_segment; - BufferHandle handle; - idx_t group_idx = 0; - uint16_t packed_data[PatasPrimitives::PATAS_GROUP_SIZE]; - - // Ptr to next free spot in segment; - data_ptr_t segment_data; - data_ptr_t metadata_ptr; - uint32_t next_group_byte_index_start = PatasPrimitives::HEADER_SIZE; - // The total size of metadata in the current segment - idx_t metadata_byte_size = 0; - - PatasState state; - -public: - idx_t RequiredSpace() const { - idx_t required_space = sizeof(EXACT_TYPE); - // byte offset of data - required_space += sizeof(byte_index_t); - // byte size of the packed_data_block - required_space += sizeof(uint16_t); - return required_space; - } - - // How many bytes the data occupies for the current segment - idx_t UsedSpace() const { - return state.patas_state.byte_writer.BytesWritten(); - } - - idx_t RemainingSpace() const { - return metadata_ptr - (handle.Ptr() + UsedSpace()); - } - - idx_t CurrentGroupMetadataSize() const { - idx_t metadata_size = 0; - - metadata_size += sizeof(byte_index_t); - metadata_size += sizeof(uint16_t) * group_idx; - return metadata_size; - } - - // The current segment has enough space to fit this new value - bool HasEnoughSpace() { - if (handle.Ptr() + AlignValue(PatasPrimitives::HEADER_SIZE + UsedSpace() + RequiredSpace()) >= - (metadata_ptr - CurrentGroupMetadataSize())) { - return false; - } - return true; - } - - void CreateEmptySegment(idx_t row_start) { - next_group_byte_index_start = PatasPrimitives::HEADER_SIZE; - group_idx = 0; - metadata_byte_size = 0; - auto &db = checkpointer.GetDatabase(); - auto &type = checkpointer.GetType(); - auto compressed_segment = ColumnSegment::CreateTransientSegment(db, type, row_start); - compressed_segment->function = function; - current_segment = std::move(compressed_segment); - - auto &buffer_manager = BufferManager::GetBufferManager(db); - handle = buffer_manager.Pin(current_segment->block); - - segment_data = handle.Ptr() + PatasPrimitives::HEADER_SIZE; - metadata_ptr = handle.Ptr() + Storage::BLOCK_SIZE; - state.AssignDataBuffer(segment_data); - state.patas_state.Reset(); - } - - void Append(UnifiedVectorFormat &vdata, idx_t count) { - auto data = UnifiedVectorFormat::GetData(vdata); - - for (idx_t i = 0; i < count; i++) { - auto idx = vdata.sel->get_index(i); - state.template Update(data[idx], vdata.validity.RowIsValid(idx)); - } - } - - void WriteValue(EXACT_TYPE value) { - current_segment->count++; - patas::PatasCompression::Store(value, state.patas_state); - group_idx++; - if (group_idx == PatasPrimitives::PATAS_GROUP_SIZE) { - FlushGroup(); - } - } - - void FlushGroup() { - metadata_ptr -= sizeof(byte_index_t); - metadata_byte_size += sizeof(byte_index_t); - // Store where this groups data starts, relative to the start of the segment - Store(next_group_byte_index_start, metadata_ptr); - next_group_byte_index_start = PatasPrimitives::HEADER_SIZE + UsedSpace(); - - // Store the packed data blocks (7 + 6 + 3 bits) - metadata_ptr -= group_idx * sizeof(uint16_t); - metadata_byte_size += group_idx * sizeof(uint16_t); - memcpy(metadata_ptr, packed_data, sizeof(uint16_t) * group_idx); - - state.patas_state.Reset(); - group_idx = 0; - } - - //! FIXME: only compact if the unused space meets a certain threshold (20%) - void FlushSegment() { - if (group_idx != 0) { - FlushGroup(); - } - auto &checkpoint_state = checkpointer.GetCheckpointState(); - auto dataptr = handle.Ptr(); - - // Compact the segment by moving the metadata next to the data. - idx_t bytes_used_by_data = PatasPrimitives::HEADER_SIZE + UsedSpace(); - idx_t metadata_offset = AlignValue(bytes_used_by_data); - // Verify that the metadata_ptr does not cross this threshold - D_ASSERT(dataptr + metadata_offset <= metadata_ptr); - idx_t metadata_size = dataptr + Storage::BLOCK_SIZE - metadata_ptr; - idx_t total_segment_size = metadata_offset + metadata_size; -#ifdef DEBUG - //! Copy the first 4 bytes of the metadata - uint32_t verify_bytes; - std::memcpy((void *)&verify_bytes, metadata_ptr, 4); -#endif - memmove(dataptr + metadata_offset, metadata_ptr, metadata_size); -#ifdef DEBUG - //! Now assert that the memmove was correct - D_ASSERT(verify_bytes == *(uint32_t *)(dataptr + metadata_offset)); -#endif - // Store the offset to the metadata - Store(metadata_offset + metadata_size, dataptr); - handle.Destroy(); - checkpoint_state.FlushSegment(std::move(current_segment), total_segment_size); - } - - void Finalize() { - FlushSegment(); - current_segment.reset(); - } -}; +struct PatasCompressionState : public CompressionState {}; // Compression Functions template unique_ptr PatasInitCompression(ColumnDataCheckpointer &checkpointer, unique_ptr state) { - return make_uniq>(checkpointer, (PatasAnalyzeState *)state.get()); + throw InternalException("Patas has been deprecated, can no longer be used to compress data"); + return nullptr; } template void PatasCompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { - auto &state = (PatasCompressionState &)state_p; - UnifiedVectorFormat vdata; - scan_vector.ToUnifiedFormat(count, vdata); - state.Append(vdata, count); + throw InternalException("Patas has been deprecated, can no longer be used to compress data"); } template void PatasFinalizeCompress(CompressionState &state_p) { - auto &state = (PatasCompressionState &)state_p; - state.Finalize(); + throw InternalException("Patas has been deprecated, can no longer be used to compress data"); } } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/data_table.hpp b/src/duckdb/src/include/duckdb/storage/data_table.hpp index b655159f3..4c2e0d6b5 100644 --- a/src/duckdb/src/include/duckdb/storage/data_table.hpp +++ b/src/duckdb/src/include/duckdb/storage/data_table.hpp @@ -147,8 +147,8 @@ class DataTable { //! Append a chunk with the row ids [row_start, ..., row_start + chunk.size()] to all indexes of the table, returns //! whether or not the append succeeded - PreservedError AppendToIndexes(DataChunk &chunk, row_t row_start); - static PreservedError AppendToIndexes(TableIndexList &indexes, DataChunk &chunk, row_t row_start); + ErrorData AppendToIndexes(DataChunk &chunk, row_t row_start); + static ErrorData AppendToIndexes(TableIndexList &indexes, DataChunk &chunk, row_t row_start); //! Remove a chunk with the row ids [row_start, ..., row_start + chunk.size()] from all indexes of the table void RemoveFromIndexes(TableAppendState &state, DataChunk &chunk, row_t row_start); //! Remove the chunk with the specified set of row identifiers from all indexes of the table diff --git a/src/duckdb/src/include/duckdb/storage/database_size.hpp b/src/duckdb/src/include/duckdb/storage/database_size.hpp index fa2353e59..79dee4bc8 100644 --- a/src/duckdb/src/include/duckdb/storage/database_size.hpp +++ b/src/duckdb/src/include/duckdb/storage/database_size.hpp @@ -9,6 +9,7 @@ #pragma once #include "duckdb/common/common.hpp" +#include "duckdb/storage/storage_info.hpp" namespace duckdb { diff --git a/src/duckdb/src/include/duckdb/storage/index.hpp b/src/duckdb/src/include/duckdb/storage/index.hpp index 6ccb08300..7fb6ffbdf 100644 --- a/src/duckdb/src/include/duckdb/storage/index.hpp +++ b/src/duckdb/src/include/duckdb/storage/index.hpp @@ -59,26 +59,17 @@ class Index { AttachedDatabase &db; public: - //! Initialize a single predicate scan on the index with the given expression and column IDs - virtual unique_ptr InitializeScanSinglePredicate(const Transaction &transaction, const Value &value, - const ExpressionType expression_type) = 0; - //! Initialize a two predicate scan on the index with the given expression and column IDs - virtual unique_ptr InitializeScanTwoPredicates(const Transaction &transaction, - const Value &low_value, - const ExpressionType low_expression_type, - const Value &high_value, - const ExpressionType high_expression_type) = 0; - //! Performs a lookup on the index, fetching up to max_count result IDs. Returns true if all row IDs were fetched, - //! and false otherwise - virtual bool Scan(const Transaction &transaction, const DataTable &table, IndexScanState &state, - const idx_t max_count, vector &result_ids) = 0; + //! Returns true if the index is a unknown index, and false otherwise + virtual bool IsUnknown() { + return false; + } //! Obtain a lock on the index void InitializeLock(IndexLock &state); //! Called when data is appended to the index. The lock obtained from InitializeLock must be held - virtual PreservedError Append(IndexLock &state, DataChunk &entries, Vector &row_identifiers) = 0; + virtual ErrorData Append(IndexLock &state, DataChunk &entries, Vector &row_identifiers) = 0; //! Obtains a lock and calls Append while holding that lock - PreservedError Append(DataChunk &entries, Vector &row_identifiers); + ErrorData Append(DataChunk &entries, Vector &row_identifiers); //! Verify that data can be appended to the index without a constraint violation virtual void VerifyAppend(DataChunk &chunk) = 0; //! Verify that data can be appended to the index without a constraint violation using the conflict manager @@ -96,7 +87,7 @@ class Index { void Delete(DataChunk &entries, Vector &row_identifiers); //! Insert a chunk of entries into the index - virtual PreservedError Insert(IndexLock &lock, DataChunk &input, Vector &row_identifiers) = 0; + virtual ErrorData Insert(IndexLock &lock, DataChunk &input, Vector &row_identifiers) = 0; //! Merge another index into this index. The lock obtained from InitializeLock must be held, and the other //! index must also be locked during the merge @@ -143,6 +134,10 @@ class Index { void ExecuteExpressions(DataChunk &input, DataChunk &result); static string AppendRowError(DataChunk &input, idx_t index); + //! Throw a constraint violation exception + virtual string GetConstraintViolationMessage(VerifyExistenceType verify_type, idx_t failed_index, + DataChunk &input) = 0; + protected: //! Lock used for any changes to the index mutex lock; diff --git a/src/duckdb/src/include/duckdb/storage/metadata/metadata_manager.hpp b/src/duckdb/src/include/duckdb/storage/metadata/metadata_manager.hpp index abe947356..900dde009 100644 --- a/src/duckdb/src/include/duckdb/storage/metadata/metadata_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/metadata/metadata_manager.hpp @@ -42,10 +42,10 @@ struct MetadataHandle { class MetadataManager { public: - //! The size of metadata blocks - static constexpr const idx_t METADATA_BLOCK_SIZE = 4088; //! The amount of metadata blocks per storage block static constexpr const idx_t METADATA_BLOCK_COUNT = 64; + //! The size of metadata blocks + static constexpr const idx_t METADATA_BLOCK_SIZE = AlignValueFloor(Storage::BLOCK_SIZE / METADATA_BLOCK_COUNT); public: MetadataManager(BlockManager &block_manager, BufferManager &buffer_manager); @@ -88,4 +88,8 @@ class MetadataManager { void ConvertToTransient(MetadataBlock &block); }; +//! Detect mismatching constant values +static_assert(MetadataManager::METADATA_BLOCK_SIZE * MetadataManager::METADATA_BLOCK_COUNT <= Storage::BLOCK_SIZE, + "metadata block count exceeds total block alloc size"); + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/object_cache.hpp b/src/duckdb/src/include/duckdb/storage/object_cache.hpp index 170c0041f..5136762db 100644 --- a/src/duckdb/src/include/duckdb/storage/object_cache.hpp +++ b/src/duckdb/src/include/duckdb/storage/object_cache.hpp @@ -16,7 +16,6 @@ #include "duckdb/main/database.hpp" namespace duckdb { -class ClientContext; //! ObjectCache is the base class for objects caches in DuckDB class ObjectCacheEntry { diff --git a/src/duckdb/src/include/duckdb/storage/segment/uncompressed.hpp b/src/duckdb/src/include/duckdb/storage/segment/uncompressed.hpp index 661d3f0fd..04876becb 100644 --- a/src/duckdb/src/include/duckdb/storage/segment/uncompressed.hpp +++ b/src/duckdb/src/include/duckdb/storage/segment/uncompressed.hpp @@ -40,9 +40,14 @@ struct StringUncompressed { static CompressionFunction GetFunction(PhysicalType data_type); public: - //! The max string size that is allowed within a block. Strings bigger than this will be labeled as a BIG STRING and - //! offloaded to the overflow blocks. - static constexpr uint16_t STRING_BLOCK_LIMIT = 4096; + //! The default maximum string size for sufficiently big block sizes + static constexpr uint16_t DEFAULT_STRING_BLOCK_LIMIT = 4096; + //! The maximum string size within a block. We offload bigger strings to overflow blocks + static constexpr uint16_t STRING_BLOCK_LIMIT = + MinValue(AlignValueFloor(Storage::BLOCK_SIZE / 4), idx_t(DEFAULT_STRING_BLOCK_LIMIT)); }; +//! Detect mismatching constant values +static_assert(StringUncompressed::STRING_BLOCK_LIMIT != 0, "the string block limit cannot be 0"); + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/standard_buffer_manager.hpp b/src/duckdb/src/include/duckdb/storage/standard_buffer_manager.hpp index ff867ef59..1b63d38fe 100644 --- a/src/duckdb/src/include/duckdb/storage/standard_buffer_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/standard_buffer_manager.hpp @@ -13,13 +13,14 @@ #include "duckdb/common/file_system.hpp" #include "duckdb/common/mutex.hpp" #include "duckdb/storage/block_manager.hpp" - #include "duckdb/storage/buffer/block_handle.hpp" -#include "duckdb/storage/buffer_manager.hpp" #include "duckdb/storage/buffer/buffer_pool.hpp" +#include "duckdb/storage/buffer_manager.hpp" namespace duckdb { + class BlockManager; +class TemporaryMemoryManager; class DatabaseInstance; class TemporaryDirectoryHandle; struct EvictionQueue; @@ -109,7 +110,8 @@ class StandardBufferManager : public BufferManager { //! Garbage collect eviction queue void PurgeQueue() final override; - BufferPool &GetBufferPool() final override; + BufferPool &GetBufferPool() const final override; + TemporaryMemoryManager &GetTemporaryMemoryManager() final override; //! Write a temporary buffer to disk void WriteTemporaryBuffer(block_id_t block_id, FileBuffer &buffer) final override; diff --git a/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats_union.hpp b/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats_union.hpp index 8006c2060..2c43987ab 100644 --- a/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats_union.hpp +++ b/src/duckdb/src/include/duckdb/storage/statistics/numeric_stats_union.hpp @@ -26,6 +26,7 @@ struct NumericValueUnion { uint32_t uinteger; uint64_t ubigint; hugeint_t hugeint; + uhugeint_t uhugeint; float float_; double double_; } value_; @@ -47,6 +48,8 @@ DUCKDB_API int64_t &NumericValueUnion::GetReferenceUnsafe(); template <> DUCKDB_API hugeint_t &NumericValueUnion::GetReferenceUnsafe(); template <> +DUCKDB_API uhugeint_t &NumericValueUnion::GetReferenceUnsafe(); +template <> DUCKDB_API uint8_t &NumericValueUnion::GetReferenceUnsafe(); template <> DUCKDB_API uint16_t &NumericValueUnion::GetReferenceUnsafe(); diff --git a/src/duckdb/src/include/duckdb/storage/storage_info.hpp b/src/duckdb/src/include/duckdb/storage/storage_info.hpp index 60ea393f2..3fa6ffc01 100644 --- a/src/duckdb/src/include/duckdb/storage/storage_info.hpp +++ b/src/duckdb/src/include/duckdb/storage/storage_info.hpp @@ -11,32 +11,35 @@ #include "duckdb/common/constants.hpp" #include "duckdb/common/string.hpp" #include "duckdb/common/vector_size.hpp" +#include "duckdb/common/limits.hpp" namespace duckdb { struct FileHandle; +//! The standard row group size #define STANDARD_ROW_GROUPS_SIZE 122880 -#if STANDARD_ROW_GROUPS_SIZE < STANDARD_VECTOR_SIZE -#error Row groups should be able to hold at least one vector -#endif +//! The definition of an invalid block +#define INVALID_BLOCK (-1) +//! The maximum block id is 2^62 +#define MAXIMUM_BLOCK 4611686018427388000LL +//! The default block size +#define DEFAULT_BLOCK_ALLOC_SIZE 262144 -#if ((STANDARD_ROW_GROUPS_SIZE % STANDARD_VECTOR_SIZE) != 0) -#error Row group size should be cleanly divisible by vector size -#endif +using block_id_t = int64_t; struct Storage { //! The size of a hard disk sector, only really needed for Direct IO - constexpr static int SECTOR_SIZE = 4096; + constexpr static idx_t SECTOR_SIZE = 4096; //! Block header size for blocks written to the storage - constexpr static int BLOCK_HEADER_SIZE = sizeof(uint64_t); - // Size of a memory slot managed by the StorageManager. This is the quantum of allocation for Blocks on DuckDB. We - // default to 256KB. (1 << 18) - constexpr static int BLOCK_ALLOC_SIZE = 262144; + constexpr static idx_t BLOCK_HEADER_SIZE = sizeof(uint64_t); + //! Size of a memory slot managed by the StorageManager. This is the quantum of allocation for Blocks on DuckDB. We + //! default to 256KB. (1 << 18) + constexpr static idx_t BLOCK_ALLOC_SIZE = DEFAULT_BLOCK_ALLOC_SIZE; //! The actual memory space that is available within the blocks - constexpr static int BLOCK_SIZE = BLOCK_ALLOC_SIZE - BLOCK_HEADER_SIZE; + constexpr static idx_t BLOCK_SIZE = BLOCK_ALLOC_SIZE - BLOCK_HEADER_SIZE; //! The size of the headers. This should be small and written more or less atomically by the hard disk. We default //! to the page size, which is 4KB. (1 << 12) - constexpr static int FILE_HEADER_SIZE = 4096; + constexpr static idx_t FILE_HEADER_SIZE = 4096; //! The number of rows per row group (must be a multiple of the vector size) constexpr static const idx_t ROW_GROUP_SIZE = STANDARD_ROW_GROUPS_SIZE; //! The number of vectors per row group @@ -45,16 +48,8 @@ struct Storage { //! The version number of the database storage format extern const uint64_t VERSION_NUMBER; - const char *GetDuckDBVersion(idx_t version_number); -using block_id_t = int64_t; - -#define INVALID_BLOCK (-1) - -// maximum block id, 2^62 -#define MAXIMUM_BLOCK 4611686018427388000LL - //! The MainHeader is the first header in the storage file. The MainHeader is typically written only once for a database //! file. struct MainHeader { @@ -62,8 +57,7 @@ struct MainHeader { static constexpr idx_t MAGIC_BYTE_SIZE = 4; static constexpr idx_t MAGIC_BYTE_OFFSET = Storage::BLOCK_HEADER_SIZE; static constexpr idx_t FLAG_COUNT = 4; - // the magic bytes in front of the file - // should be "DUCK" + //! The magic bytes in front of the file should be "DUCK" static const char MAGIC_BYTES[]; //! The version of the database uint64_t version_number; @@ -100,9 +94,28 @@ struct DatabaseHeader { //! The number of blocks that is in the file as of this database header. If the file is larger than BLOCK_SIZE * //! block_count any blocks appearing AFTER block_count are implicitly part of the free_list. uint64_t block_count; + //! The block size of the database file + idx_t block_size; + //! The vector size of the database file + idx_t vector_size; void Write(WriteStream &ser); static DatabaseHeader Read(ReadStream &source); }; +//! Detect mismatching constant values when compiling + +#if (STANDARD_ROW_GROUPS_SIZE % STANDARD_VECTOR_SIZE != 0) +#error The row group size must be a multiple of the vector size +#endif +#if (STANDARD_ROW_GROUPS_SIZE < STANDARD_VECTOR_SIZE) +#error Row groups must be able to hold at least one vector +#endif + +static_assert(Storage::BLOCK_ALLOC_SIZE % Storage::SECTOR_SIZE == 0, + "the block allocation size has to be a multiple of the sector size"); +static_assert(Storage::BLOCK_SIZE < idx_t(NumericLimits::Maximum()), + "the block size cannot exceed the maximum signed integer value," + "as some comparisons require casts"); + } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/storage_manager.hpp b/src/duckdb/src/include/duckdb/storage/storage_manager.hpp index 7bd92c29f..5c063c3fe 100644 --- a/src/duckdb/src/include/duckdb/storage/storage_manager.hpp +++ b/src/duckdb/src/include/duckdb/storage/storage_manager.hpp @@ -54,13 +54,14 @@ class StorageManager { } //! Get the WAL of the StorageManager, returns nullptr if in-memory - optional_ptr GetWriteAheadLog() { - return wal.get(); - } + optional_ptr GetWriteAheadLog(); + //! Returns the database file path string GetDBPath() { return path; } + //! The path to the WAL, derived from the database file path + string GetWALPath(); bool InMemory(); virtual bool AutomaticCheckpoint(idx_t estimated_wal_bytes) = 0; @@ -75,7 +76,7 @@ class StorageManager { virtual void LoadDatabase() = 0; protected: - //! The database this storagemanager belongs to + //! The database this storage manager belongs to AttachedDatabase &db; //! The path of the database string path; @@ -83,6 +84,9 @@ class StorageManager { unique_ptr wal; //! Whether or not the database is opened in read-only mode bool read_only; + //! When loading a database, we do not yet set the wal-field. Therefore, GetWriteAheadLog must + //! return nullptr when loading a database + bool load_complete = false; public: template 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 index d931067b1..2e6760453 100644 --- a/src/duckdb/src/include/duckdb/storage/table/array_column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/array_column_data.hpp @@ -56,7 +56,7 @@ class ArrayColumnData : public ColumnData { unique_ptr Checkpoint(RowGroup &row_group, PartialBlockManager &partial_block_manager, ColumnCheckpointInfo &checkpoint_info) override; - void DeserializeColumn(Deserializer &source) override; + void DeserializeColumn(Deserializer &source, BaseStatistics &target_stats) override; void GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, vector &result) override; diff --git a/src/duckdb/src/include/duckdb/storage/table/column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/column_data.hpp index c278b3a54..c555b2523 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_data.hpp @@ -26,6 +26,7 @@ class RowGroupWriter; class TableDataWriter; class TableStorageInfo; struct TransactionData; +struct TableScanOptions; struct DataTableInfo; @@ -125,10 +126,9 @@ class ColumnData { virtual void CheckpointScan(ColumnSegment &segment, ColumnScanState &state, idx_t row_group_start, idx_t count, Vector &scan_vector); - virtual void DeserializeColumn(Deserializer &deserializer); + virtual void DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats); static shared_ptr Deserialize(BlockManager &block_manager, DataTableInfo &info, idx_t column_index, - idx_t start_row, ReadStream &source, const LogicalType &type, - optional_ptr parent); + idx_t start_row, ReadStream &source, const LogicalType &type); virtual void GetColumnSegmentInfo(idx_t row_group_index, vector col_path, vector &result); virtual void Verify(RowGroup &parent); diff --git a/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp b/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp index aedb2ee81..eb105e55a 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_data_checkpointer.hpp @@ -13,6 +13,7 @@ #include "duckdb/storage/table/column_checkpoint_state.hpp" namespace duckdb { +struct TableScanOptions; class ColumnDataCheckpointer { public: diff --git a/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp b/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp index 0ca098400..6ae03d4f3 100644 --- a/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/column_segment.hpp @@ -60,7 +60,6 @@ class ColumnSegment : public SegmentBase { unique_ptr segment_state); static unique_ptr CreateTransientSegment(DatabaseInstance &db, const LogicalType &type, idx_t start, idx_t segment_size = Storage::BLOCK_SIZE); - static unique_ptr CreateSegment(ColumnSegment &other, idx_t start); public: void InitializeScan(ColumnScanState &state); @@ -140,7 +139,7 @@ class ColumnSegment : public SegmentBase { block_id_t block_id; //! The offset into the block (persistent segment only) idx_t offset; - //! The allocated segment size + //! The allocated segment size, which is bounded by Storage::BLOCK_SIZE idx_t segment_size; //! Storage associated with the compressed segment unique_ptr segment_state; diff --git a/src/duckdb/src/include/duckdb/storage/table/data_table_info.hpp b/src/duckdb/src/include/duckdb/storage/table/data_table_info.hpp index 7bf732f13..2785486b5 100644 --- a/src/duckdb/src/include/duckdb/storage/table/data_table_info.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/data_table_info.hpp @@ -19,6 +19,9 @@ class TableIOManager; struct DataTableInfo { DataTableInfo(AttachedDatabase &db, shared_ptr table_io_manager_p, string schema, string table); + //! Initialize any unknown indexes whose types might now be present after an extension load + void InitializeIndexes(ClientContext &context); + //! The database instance of the table AttachedDatabase &db; //! The table IO manager diff --git a/src/duckdb/src/include/duckdb/storage/table/list_column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/list_column_data.hpp index 6001e7036..3140ea37a 100644 --- a/src/duckdb/src/include/duckdb/storage/table/list_column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/list_column_data.hpp @@ -56,7 +56,7 @@ class ListColumnData : public ColumnData { unique_ptr Checkpoint(RowGroup &row_group, PartialBlockManager &partial_block_manager, ColumnCheckpointInfo &checkpoint_info) override; - void DeserializeColumn(Deserializer &deserializer) override; + void DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) override; void GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, vector &result) override; diff --git a/src/duckdb/src/include/duckdb/storage/table/scan_state.hpp b/src/duckdb/src/include/duckdb/storage/table/scan_state.hpp index cefa13a27..56dfc8be5 100644 --- a/src/duckdb/src/include/duckdb/storage/table/scan_state.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/scan_state.hpp @@ -31,6 +31,7 @@ class TableFilterSet; class ColumnData; class DuckTransaction; class RowGroupSegmentTree; +struct TableScanOptions; struct SegmentScanState { virtual ~SegmentScanState() { @@ -92,9 +93,11 @@ struct ColumnScanState { vector> previous_states; //! The last read offset in the child state (used for LIST columns only) idx_t last_offset = 0; + //! Contains TableScan level config for scanning + optional_ptr scan_options; public: - void Initialize(const LogicalType &type); + void Initialize(const LogicalType &type, optional_ptr options); //! Move the scan state forward by "count" rows (including all child states) void Next(idx_t count); //! Move ONLY this state forward by "count" rows (i.e. not the child states) @@ -134,6 +137,7 @@ class CollectionScanState { const vector &GetColumnIds(); TableFilterSet *GetFilters(); AdaptiveFilter *GetAdaptiveFilter(); + TableScanOptions &GetOptions(); bool Scan(DuckTransaction &transaction, DataChunk &result); bool ScanCommitted(DataChunk &result, TableScanType type); bool ScanCommitted(DataChunk &result, SegmentLock &l, TableScanType type); @@ -142,6 +146,11 @@ class CollectionScanState { TableScanState &parent; }; +struct TableScanOptions { + //! Test config that forces fetching rows one by one instead of regular scans + bool force_fetch_row = false; +}; + class TableScanState { public: TableScanState() : table_state(*this), local_state(*this), table_filters(nullptr) {}; @@ -150,6 +159,8 @@ class TableScanState { CollectionScanState table_state; //! Transaction-local scan state CollectionScanState local_state; + //! Options for scanning + TableScanOptions options; public: void Initialize(vector column_ids, TableFilterSet *table_filters = nullptr); diff --git a/src/duckdb/src/include/duckdb/storage/table/standard_column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/standard_column_data.hpp index 17bd5eebe..135024122 100644 --- a/src/duckdb/src/include/duckdb/storage/table/standard_column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/standard_column_data.hpp @@ -57,7 +57,7 @@ class StandardColumnData : public ColumnData { void GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, vector &result) override; - void DeserializeColumn(Deserializer &deserializer) override; + void DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) override; void Verify(RowGroup &parent) override; }; diff --git a/src/duckdb/src/include/duckdb/storage/table/struct_column_data.hpp b/src/duckdb/src/include/duckdb/storage/table/struct_column_data.hpp index 61956b1f7..723b2b9f4 100644 --- a/src/duckdb/src/include/duckdb/storage/table/struct_column_data.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/struct_column_data.hpp @@ -57,7 +57,7 @@ class StructColumnData : public ColumnData { unique_ptr Checkpoint(RowGroup &row_group, PartialBlockManager &partial_block_manager, ColumnCheckpointInfo &checkpoint_info) override; - void DeserializeColumn(Deserializer &source) override; + void DeserializeColumn(Deserializer &source, BaseStatistics &target_stats) override; void GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, vector &result) override; diff --git a/src/duckdb/src/include/duckdb/storage/table/table_index_list.hpp b/src/duckdb/src/include/duckdb/storage/table/table_index_list.hpp index fd3c647b7..be15497ad 100644 --- a/src/duckdb/src/include/duckdb/storage/table/table_index_list.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/table_index_list.hpp @@ -15,6 +15,7 @@ namespace duckdb { class ConflictManager; struct IndexStorageInfo; +struct DataTableInfo; class TableIndexList { public: @@ -40,7 +41,8 @@ class TableIndexList { void CommitDrop(const string &name); //! Returns true, if the index name does not exist bool NameIsUnique(const string &name); - + //! Initializes unknown indexes that might now be present after an extension load + void InitializeIndexes(ClientContext &context, DataTableInfo &table_info); bool Empty(); idx_t Count(); void Move(TableIndexList &other); diff --git a/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp b/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp index 52b0ec9a8..b23253278 100644 --- a/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp +++ b/src/duckdb/src/include/duckdb/storage/table/table_statistics.hpp @@ -10,6 +10,7 @@ #include "duckdb/common/common.hpp" #include "duckdb/common/types/data_chunk.hpp" +#include "duckdb/execution/reservoir_sample.hpp" #include "duckdb/common/mutex.hpp" #include "duckdb/storage/statistics/column_statistics.hpp" @@ -57,6 +58,9 @@ class TableStatistics { mutex stats_lock; //! Column statistics vector> column_stats; + //! The table sample + //! Sample for table + unique_ptr table_sample; }; } // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/temporary_memory_manager.hpp b/src/duckdb/src/include/duckdb/storage/temporary_memory_manager.hpp new file mode 100644 index 000000000..01576a334 --- /dev/null +++ b/src/duckdb/src/include/duckdb/storage/temporary_memory_manager.hpp @@ -0,0 +1,117 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/storage/temporary_memory_manager.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/common/atomic.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/reference_map.hpp" +#include "duckdb/storage/storage_info.hpp" + +namespace duckdb { + +class ClientContext; +class TemporaryMemoryManager; + +//! State of the temporary memory to be managed concurrently with other states +//! As long as this is within scope, it is active +class TemporaryMemoryState { + friend class TemporaryMemoryManager; + +private: + TemporaryMemoryState(TemporaryMemoryManager &temporary_memory_manager, idx_t minimum_reservation); + +public: + ~TemporaryMemoryState(); + +public: + //! Set the remaining size needed for this state, and updates the reservation + void SetRemainingSize(ClientContext &context, idx_t new_remaining_size); + //! Get the remaining size that was set for this state + idx_t GetRemainingSize() const; + //! Set the minimum reservation for this state + void SetMinimumReservation(idx_t new_minimum_reservation); + //! Get the reservation of this state + idx_t GetReservation() const; + +private: + //! The TemporaryMemoryManager that owns this state + TemporaryMemoryManager &temporary_memory_manager; + + //! The remaining size needed if it could fit fully in memory + atomic remaining_size; + //! The minimum reservation for this state + atomic minimum_reservation; + //! How much memory this operator has reserved + atomic reservation; +}; + +//! TemporaryMemoryManager is a one-of class owned by the buffer pool that tries to dynamically assign memory +//! to concurrent states, such that their combined memory usage does not exceed the limit +class TemporaryMemoryManager { + //! TemporaryMemoryState is a friend class so it can access the private methods of this class, + //! but it should not access the private fields! + friend class TemporaryMemoryState; + +public: + TemporaryMemoryManager(); + +private: + //! TemporaryMemoryState is initialized with a minimum reservation guarantee, which is either + //! 512 blocks per state per thread, which is 0.125GB per thread for Storage::BLOCK_ALLOC_SIZE = 262144 + static constexpr const idx_t MINIMUM_RESERVATION_PER_STATE_PER_THREAD = idx_t(512) * Storage::BLOCK_ALLOC_SIZE; + //! Or 1/16th% of main memory, if that is lower + static constexpr const idx_t MINIMUM_RESERVATION_MEMORY_LIMIT_DIVISOR = 16; + + //! The maximum ratio of the memory limit that we reserve using the TemporaryMemoryManager + static constexpr const double MAXIMUM_MEMORY_LIMIT_RATIO = 0.8; + //! The maximum ratio of the remaining memory that we reserve per TemporaryMemoryState + static constexpr const double MAXIMUM_FREE_MEMORY_RATIO = 0.6; + +public: + //! Get the TemporaryMemoryManager + static TemporaryMemoryManager &Get(ClientContext &context); + //! Register a TemporaryMemoryState + unique_ptr Register(ClientContext &context); + +private: + //! Locks the TemporaryMemoryManager + unique_lock Lock(); + //! Update memory_limit, has_temporary_directory, and num_threads (must hold the lock) + void UpdateConfiguration(ClientContext &context); + //! Update the TemporaryMemoryState to the new remaining size, and updates the reservation (must hold the lock) + void UpdateState(ClientContext &context, TemporaryMemoryState &temporary_memory_state); + //! Set the remaining size of a TemporaryMemoryState (must hold the lock) + void SetRemainingSize(TemporaryMemoryState &temporary_memory_state, idx_t new_remaining_size); + //! Set the reservation of a TemporaryMemoryState (must hold the lock) + void SetReservation(TemporaryMemoryState &temporary_memory_state, idx_t new_reservation); + //! Unregister a TemporaryMemoryState (called by the destructor of TemporaryMemoryState) + void Unregister(TemporaryMemoryState &temporary_memory_state); + //! Verify internal counts (must hold the lock) + void Verify() const; + +private: + //! Lock because TemporaryMemoryManager is used concurrently + mutex lock; + + //! Memory limit of the buffer pool + idx_t memory_limit; + //! Whether there is a temporary directory that we can offload blocks to + bool has_temporary_directory; + //! Number of threads + idx_t num_threads; + + //! Currently active states + reference_set_t active_states; + //! The sum of reservations of all active states + idx_t reservation; + //! The sum of the remaining size of all active states + idx_t remaining_size; +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/storage/write_ahead_log.hpp b/src/duckdb/src/include/duckdb/storage/write_ahead_log.hpp index 194ae68f7..11a11c25e 100644 --- a/src/duckdb/src/include/duckdb/storage/write_ahead_log.hpp +++ b/src/duckdb/src/include/duckdb/storage/write_ahead_log.hpp @@ -35,56 +35,7 @@ class TypeCatalogEntry; class TableCatalogEntry; class Transaction; class TransactionManager; - -class ReplayState { -public: - ReplayState(AttachedDatabase &db, ClientContext &context) - : db(db), context(context), catalog(db.GetCatalog()), deserialize_only(false) { - } - - AttachedDatabase &db; - ClientContext &context; - Catalog &catalog; - optional_ptr current_table; - bool deserialize_only; - MetaBlockPointer checkpoint_id; - -public: - void ReplayEntry(WALType entry_type, BinaryDeserializer &deserializer); - -protected: - virtual void ReplayCreateTable(BinaryDeserializer &deserializer); - void ReplayDropTable(BinaryDeserializer &deserializer); - void ReplayAlter(BinaryDeserializer &deserializer); - - void ReplayCreateView(BinaryDeserializer &deserializer); - void ReplayDropView(BinaryDeserializer &deserializer); - - void ReplayCreateSchema(BinaryDeserializer &deserializer); - void ReplayDropSchema(BinaryDeserializer &deserializer); - - void ReplayCreateType(BinaryDeserializer &deserializer); - void ReplayDropType(BinaryDeserializer &deserializer); - - void ReplayCreateSequence(BinaryDeserializer &deserializer); - void ReplayDropSequence(BinaryDeserializer &deserializer); - void ReplaySequenceValue(BinaryDeserializer &deserializer); - - void ReplayCreateMacro(BinaryDeserializer &deserializer); - void ReplayDropMacro(BinaryDeserializer &deserializer); - - void ReplayCreateTableMacro(BinaryDeserializer &deserializer); - void ReplayDropTableMacro(BinaryDeserializer &deserializer); - - void ReplayCreateIndex(BinaryDeserializer &deserializer); - void ReplayDropIndex(BinaryDeserializer &deserializer); - - void ReplayUseTable(BinaryDeserializer &deserializer); - void ReplayInsert(BinaryDeserializer &deserializer); - void ReplayDelete(BinaryDeserializer &deserializer); - void ReplayUpdate(BinaryDeserializer &deserializer); - void ReplayCheckpoint(BinaryDeserializer &deserializer); -}; +class WriteAheadLogDeserializer; //! The WriteAheadLog (WAL) is a log that is used to provide durability. Prior //! to committing a transaction it writes the changes the transaction made to @@ -108,6 +59,12 @@ class WriteAheadLog { //! Gets the total bytes written to the WAL since startup idx_t GetTotalWritten(); + BufferedFileWriter &GetWriter() { + return *writer; + } + + void WriteVersion(); + virtual void WriteCreateTable(const TableCatalogEntry &entry); void WriteDropTable(const TableCatalogEntry &entry); diff --git a/src/duckdb/src/include/duckdb/transaction/duck_transaction.hpp b/src/duckdb/src/include/duckdb/transaction/duck_transaction.hpp index 8dc116a43..858072650 100644 --- a/src/duckdb/src/include/duckdb/transaction/duck_transaction.hpp +++ b/src/duckdb/src/include/duckdb/transaction/duck_transaction.hpp @@ -39,7 +39,7 @@ class DuckTransaction : public Transaction { //! Commit the current transaction with the given commit identifier. Returns an error message if the transaction //! commit failed, or an empty string if the commit was sucessful - string Commit(AttachedDatabase &db, transaction_t commit_id, bool checkpoint) noexcept; + ErrorData Commit(AttachedDatabase &db, transaction_t commit_id, bool checkpoint) noexcept; //! Returns whether or not a commit of this transaction should trigger an automatic checkpoint bool AutomaticCheckpoint(AttachedDatabase &db); diff --git a/src/duckdb/src/include/duckdb/transaction/duck_transaction_manager.hpp b/src/duckdb/src/include/duckdb/transaction/duck_transaction_manager.hpp index 269167f52..49596e641 100644 --- a/src/duckdb/src/include/duckdb/transaction/duck_transaction_manager.hpp +++ b/src/duckdb/src/include/duckdb/transaction/duck_transaction_manager.hpp @@ -28,7 +28,7 @@ class DuckTransactionManager : public TransactionManager { //! Start a new transaction Transaction &StartTransaction(ClientContext &context) override; //! Commit the given transaction - string CommitTransaction(ClientContext &context, Transaction &transaction) override; + ErrorData CommitTransaction(ClientContext &context, Transaction &transaction) override; //! Rollback the given transaction void RollbackTransaction(Transaction &transaction) override; diff --git a/src/duckdb/src/include/duckdb/transaction/local_storage.hpp b/src/duckdb/src/include/duckdb/transaction/local_storage.hpp index 099507b5a..7481abd70 100644 --- a/src/duckdb/src/include/duckdb/transaction/local_storage.hpp +++ b/src/duckdb/src/include/duckdb/transaction/local_storage.hpp @@ -61,8 +61,8 @@ class LocalTableStorage : public std::enable_shared_from_this void AppendToIndexes(DuckTransaction &transaction, TableAppendState &append_state, idx_t append_count, bool append_to_table); - PreservedError AppendToIndexes(DuckTransaction &transaction, RowGroupCollection &source, TableIndexList &index_list, - const vector &table_types, row_t &start_row); + ErrorData AppendToIndexes(DuckTransaction &transaction, RowGroupCollection &source, TableIndexList &index_list, + const vector &table_types, row_t &start_row); //! Creates an optimistic writer for this table OptimisticDataWriter &CreateOptimisticWriter(); diff --git a/src/duckdb/src/include/duckdb/transaction/meta_transaction.hpp b/src/duckdb/src/include/duckdb/transaction/meta_transaction.hpp index 0c9b52ddb..e858b5c80 100644 --- a/src/duckdb/src/include/duckdb/transaction/meta_transaction.hpp +++ b/src/duckdb/src/include/duckdb/transaction/meta_transaction.hpp @@ -15,6 +15,7 @@ #include "duckdb/common/unordered_map.hpp" #include "duckdb/common/optional_ptr.hpp" #include "duckdb/common/reference_map.hpp" +#include "duckdb/common/error_data.hpp" namespace duckdb { class AttachedDatabase; @@ -47,7 +48,7 @@ class MetaTransaction { Transaction &GetTransaction(AttachedDatabase &db); void RemoveTransaction(AttachedDatabase &db); - string Commit(); + ErrorData Commit(); void Rollback(); idx_t GetActiveQuery(); diff --git a/src/duckdb/src/include/duckdb/transaction/transaction_manager.hpp b/src/duckdb/src/include/duckdb/transaction/transaction_manager.hpp index e18c7f620..2e37b8829 100644 --- a/src/duckdb/src/include/duckdb/transaction/transaction_manager.hpp +++ b/src/duckdb/src/include/duckdb/transaction/transaction_manager.hpp @@ -34,7 +34,7 @@ class TransactionManager { //! Start a new transaction virtual Transaction &StartTransaction(ClientContext &context) = 0; //! Commit the given transaction. Returns a non-empty error message on failure. - virtual string CommitTransaction(ClientContext &context, Transaction &transaction) = 0; + virtual ErrorData CommitTransaction(ClientContext &context, Transaction &transaction) = 0; //! Rollback the given transaction virtual void RollbackTransaction(Transaction &transaction) = 0; diff --git a/src/duckdb/src/include/duckdb/verification/fetch_row_verifier.hpp b/src/duckdb/src/include/duckdb/verification/fetch_row_verifier.hpp new file mode 100644 index 000000000..007e98657 --- /dev/null +++ b/src/duckdb/src/include/duckdb/verification/fetch_row_verifier.hpp @@ -0,0 +1,25 @@ +//===----------------------------------------------------------------------===// +// DuckDB +// +// duckdb/verification/unoptimized_statement_verifier.hpp +// +// +//===----------------------------------------------------------------------===// + +#pragma once + +#include "duckdb/verification/statement_verifier.hpp" + +namespace duckdb { + +class FetchRowVerifier : public StatementVerifier { +public: + explicit FetchRowVerifier(unique_ptr statement_p); + static unique_ptr Create(const SQLStatement &statement_p); + + bool ForceFetchRow() const override { + return true; + } +}; + +} // namespace duckdb diff --git a/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp b/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp index d14f1ffa0..27ce92a3a 100644 --- a/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp +++ b/src/duckdb/src/include/duckdb/verification/statement_verifier.hpp @@ -23,6 +23,7 @@ enum class VerificationType : uint8_t { NO_OPERATOR_CACHING, PREPARED, EXTERNAL, + FETCH_ROW_AS_SCAN, INVALID }; @@ -67,6 +68,10 @@ class StatementVerifier { virtual bool ForceExternal() const { return false; } + + virtual bool ForceFetchRow() const { + return false; + } }; } // namespace duckdb diff --git a/src/duckdb/src/main/appender.cpp b/src/duckdb/src/main/appender.cpp index 57822c383..2aa782245 100644 --- a/src/duckdb/src/main/appender.cpp +++ b/src/duckdb/src/main/appender.cpp @@ -152,6 +152,9 @@ void BaseAppender::AppendValueInternal(T input) { case LogicalTypeId::HUGEINT: AppendValueInternal(col, input); break; + case LogicalTypeId::UHUGEINT: + AppendValueInternal(col, input); + break; case LogicalTypeId::FLOAT: AppendValueInternal(col, input); break; @@ -232,6 +235,11 @@ void BaseAppender::Append(hugeint_t value) { AppendValueInternal(value); } +template <> +void BaseAppender::Append(uhugeint_t value) { + AppendValueInternal(value); +} + template <> void BaseAppender::Append(uint8_t value) { AppendValueInternal(value); @@ -319,8 +327,15 @@ void BaseAppender::AppendValue(const Value &value) { } void BaseAppender::AppendDataChunk(DataChunk &chunk) { - if (chunk.GetTypes() != types) { - throw InvalidInputException("Type mismatch in Append DataChunk and the types required for appender"); + auto chunk_types = chunk.GetTypes(); + if (chunk_types != types) { + for (idx_t i = 0; i < chunk.ColumnCount(); i++) { + if (chunk.data[i].GetType() != types[i]) { + throw InvalidInputException("Type mismatch in Append DataChunk and the types required for appender, " + "expected %s but got %s for column %d", + types[i].ToString(), chunk.data[i].GetType().ToString(), i + 1); + } + } } collection->Append(chunk); if (collection->Count() >= FLUSH_COUNT) { diff --git a/src/duckdb/src/main/capi/appender-c.cpp b/src/duckdb/src/main/capi/appender-c.cpp index 8efa299b7..95177cf2b 100644 --- a/src/duckdb/src/main/capi/appender-c.cpp +++ b/src/duckdb/src/main/capi/appender-c.cpp @@ -1,14 +1,17 @@ #include "duckdb/main/capi/capi_internal.hpp" +#include "duckdb/common/uhugeint.hpp" using duckdb::Appender; using duckdb::AppenderWrapper; using duckdb::Connection; using duckdb::date_t; using duckdb::dtime_t; +using duckdb::ErrorData; using duckdb::hugeint_t; using duckdb::interval_t; using duckdb::string_t; using duckdb::timestamp_t; +using duckdb::uhugeint_t; duckdb_state duckdb_appender_create(duckdb_connection connection, const char *schema, const char *table, duckdb_appender *out_appender) { @@ -25,7 +28,8 @@ duckdb_state duckdb_appender_create(duckdb_connection connection, const char *sc try { wrapper->appender = duckdb::make_uniq(*conn, schema, table); } catch (std::exception &ex) { - wrapper->error = ex.what(); + ErrorData error(ex); + wrapper->error = error.RawMessage(); return DuckDBError; } catch (...) { // LCOV_EXCL_START wrapper->error = "Unknown create appender error"; @@ -59,7 +63,8 @@ duckdb_state duckdb_appender_run_function(duckdb_appender appender, FUN &&functi try { function(*wrapper->appender); } catch (std::exception &ex) { - wrapper->error = ex.what(); + ErrorData error(ex); + wrapper->error = error.RawMessage(); return DuckDBError; } catch (...) { // LCOV_EXCL_START wrapper->error = "Unknown error"; @@ -96,7 +101,8 @@ duckdb_state duckdb_append_internal(duckdb_appender appender, T value) { try { appender_instance->appender->Append(value); } catch (std::exception &ex) { - appender_instance->error = ex.what(); + ErrorData error(ex); + appender_instance->error = error.RawMessage(); return DuckDBError; } catch (...) { return DuckDBError; @@ -147,6 +153,13 @@ duckdb_state duckdb_append_uint64(duckdb_appender appender, uint64_t value) { return duckdb_append_internal(appender, value); } +duckdb_state duckdb_append_uhugeint(duckdb_appender appender, duckdb_uhugeint value) { + uhugeint_t internal; + internal.lower = value.lower; + internal.upper = value.upper; + return duckdb_append_internal(appender, internal); +} + duckdb_state duckdb_append_float(duckdb_appender appender, float value) { return duckdb_append_internal(appender, value); } @@ -199,6 +212,32 @@ duckdb_state duckdb_appender_close(duckdb_appender appender) { return duckdb_appender_run_function(appender, [&](Appender &appender) { appender.Close(); }); } +idx_t duckdb_appender_column_count(duckdb_appender appender) { + if (!appender) { + return 0; + } + + auto wrapper = reinterpret_cast(appender); + if (!wrapper->appender) { + return 0; + } + + return wrapper->appender->GetTypes().size(); +} + +duckdb_logical_type duckdb_appender_column_type(duckdb_appender appender, idx_t col_idx) { + if (!appender || col_idx >= duckdb_appender_column_count(appender)) { + return nullptr; + } + + auto wrapper = reinterpret_cast(appender); + if (!wrapper->appender) { + return nullptr; + } + + return reinterpret_cast(new duckdb::LogicalType(wrapper->appender->GetTypes()[col_idx])); +} + duckdb_state duckdb_append_data_chunk(duckdb_appender appender, duckdb_data_chunk chunk) { if (!chunk) { return DuckDBError; diff --git a/src/duckdb/src/main/capi/arrow-c.cpp b/src/duckdb/src/main/capi/arrow-c.cpp index 14154954e..1b0b366a2 100644 --- a/src/duckdb/src/main/capi/arrow-c.cpp +++ b/src/duckdb/src/main/capi/arrow-c.cpp @@ -141,6 +141,21 @@ void duckdb_destroy_arrow(duckdb_arrow *result) { } } +void duckdb_destroy_arrow_stream(duckdb_arrow_stream *stream_p) { + + auto stream = reinterpret_cast(*stream_p); + if (!stream) { + return; + } + if (stream->release) { + stream->release(stream); + } + D_ASSERT(!stream->release); + + delete stream; + *stream_p = nullptr; +} + duckdb_state duckdb_execute_prepared_arrow(duckdb_prepared_statement prepared_statement, duckdb_arrow *out_result) { auto wrapper = reinterpret_cast(prepared_statement); if (!wrapper || !wrapper->statement || wrapper->statement->HasError() || !out_result) { diff --git a/src/duckdb/src/main/capi/datetime-c.cpp b/src/duckdb/src/main/capi/datetime-c.cpp index 1a3390eb7..9b07007af 100644 --- a/src/duckdb/src/main/capi/datetime-c.cpp +++ b/src/duckdb/src/main/capi/datetime-c.cpp @@ -28,6 +28,10 @@ duckdb_date duckdb_to_date(duckdb_date_struct date) { return result; } +bool duckdb_is_finite_date(duckdb_date date) { + return Date::IsFinite(date_t(date.days)); +} + duckdb_time_struct duckdb_from_time(duckdb_time time) { int32_t hour, minute, second, micros; Time::Convert(dtime_t(time.micros), hour, minute, second, micros); @@ -40,6 +44,20 @@ duckdb_time_struct duckdb_from_time(duckdb_time time) { return result; } +duckdb_time_tz_struct duckdb_from_time_tz(duckdb_time_tz input) { + duckdb::dtime_tz_t time(input.bits); + duckdb_time_tz_struct result; + result.time.micros = time.time().micros; + result.offset = time.offset(); + return result; +} + +duckdb_time_tz duckdb_create_time_tz(int64_t micros, int32_t offset) { + duckdb_time_tz time; + time.bits = duckdb::dtime_tz_t(duckdb::dtime_t(micros), offset).bits; + return time; +} + duckdb_time duckdb_to_time(duckdb_time_struct time) { duckdb_time result; result.micros = Time::FromTime(time.hour, time.min, time.sec, time.micros).micros; @@ -71,3 +89,7 @@ duckdb_timestamp duckdb_to_timestamp(duckdb_timestamp_struct ts) { result.micros = Timestamp::FromDatetime(date, time).value; return result; } + +bool duckdb_is_finite_timestamp(duckdb_timestamp ts) { + return Timestamp::IsFinite(timestamp_t(ts.micros)); +} diff --git a/src/duckdb/src/main/capi/duckdb-c.cpp b/src/duckdb/src/main/capi/duckdb-c.cpp index 48a2eb466..2b14b8178 100644 --- a/src/duckdb/src/main/capi/duckdb-c.cpp +++ b/src/duckdb/src/main/capi/duckdb-c.cpp @@ -4,6 +4,7 @@ using duckdb::Connection; using duckdb::DatabaseData; using duckdb::DBConfig; using duckdb::DuckDB; +using duckdb::ErrorData; duckdb_state duckdb_open_ext(const char *path, duckdb_database *out, duckdb_config config, char **error) { auto wrapper = new DatabaseData(); @@ -20,7 +21,8 @@ duckdb_state duckdb_open_ext(const char *path, duckdb_database *out, duckdb_conf wrapper->database = duckdb::make_uniq(path, db_config); } catch (std::exception &ex) { if (error) { - *error = strdup(ex.what()); + ErrorData parsed_error(ex); + *error = strdup(parsed_error.Message().c_str()); } delete wrapper; return DuckDBError; diff --git a/src/duckdb/src/main/capi/helper-c.cpp b/src/duckdb/src/main/capi/helper-c.cpp index c377630af..b9caf94df 100644 --- a/src/duckdb/src/main/capi/helper-c.cpp +++ b/src/duckdb/src/main/capi/helper-c.cpp @@ -24,16 +24,22 @@ LogicalTypeId ConvertCTypeToCPP(duckdb_type c_type) { return LogicalTypeId::UBIGINT; case DUCKDB_TYPE_HUGEINT: return LogicalTypeId::HUGEINT; + case DUCKDB_TYPE_UHUGEINT: + return LogicalTypeId::UHUGEINT; case DUCKDB_TYPE_FLOAT: return LogicalTypeId::FLOAT; case DUCKDB_TYPE_DOUBLE: return LogicalTypeId::DOUBLE; case DUCKDB_TYPE_TIMESTAMP: return LogicalTypeId::TIMESTAMP; + case DUCKDB_TYPE_TIMESTAMP_TZ: + return LogicalTypeId::TIMESTAMP_TZ; case DUCKDB_TYPE_DATE: return LogicalTypeId::DATE; case DUCKDB_TYPE_TIME: return LogicalTypeId::TIME; + case DUCKDB_TYPE_TIME_TZ: + return LogicalTypeId::TIME_TZ; case DUCKDB_TYPE_VARCHAR: return LogicalTypeId::VARCHAR; case DUCKDB_TYPE_BLOB: @@ -76,13 +82,16 @@ duckdb_type ConvertCPPTypeToC(const LogicalType &sql_type) { return DUCKDB_TYPE_UBIGINT; case LogicalTypeId::HUGEINT: return DUCKDB_TYPE_HUGEINT; + case LogicalTypeId::UHUGEINT: + return DUCKDB_TYPE_UHUGEINT; case LogicalTypeId::FLOAT: return DUCKDB_TYPE_FLOAT; case LogicalTypeId::DOUBLE: return DUCKDB_TYPE_DOUBLE; case LogicalTypeId::TIMESTAMP: - case LogicalTypeId::TIMESTAMP_TZ: return DUCKDB_TYPE_TIMESTAMP; + case LogicalTypeId::TIMESTAMP_TZ: + return DUCKDB_TYPE_TIMESTAMP_TZ; case LogicalTypeId::TIMESTAMP_SEC: return DUCKDB_TYPE_TIMESTAMP_S; case LogicalTypeId::TIMESTAMP_MS: @@ -92,8 +101,9 @@ duckdb_type ConvertCPPTypeToC(const LogicalType &sql_type) { case LogicalTypeId::DATE: return DUCKDB_TYPE_DATE; case LogicalTypeId::TIME: - case LogicalTypeId::TIME_TZ: return DUCKDB_TYPE_TIME; + case LogicalTypeId::TIME_TZ: + return DUCKDB_TYPE_TIME_TZ; case LogicalTypeId::VARCHAR: return DUCKDB_TYPE_VARCHAR; case LogicalTypeId::BLOB: @@ -142,6 +152,7 @@ idx_t GetCTypeSize(duckdb_type type) { return sizeof(uint32_t); case DUCKDB_TYPE_UBIGINT: return sizeof(uint64_t); + case DUCKDB_TYPE_UHUGEINT: case DUCKDB_TYPE_HUGEINT: case DUCKDB_TYPE_UUID: return sizeof(duckdb_hugeint); @@ -211,8 +222,6 @@ duckdb_statement_type StatementTypeToC(duckdb::StatementType statement_type) { return DUCKDB_STATEMENT_TYPE_EXPORT; case duckdb::StatementType::PRAGMA_STATEMENT: return DUCKDB_STATEMENT_TYPE_PRAGMA; - case duckdb::StatementType::SHOW_STATEMENT: - return DUCKDB_STATEMENT_TYPE_SHOW; case duckdb::StatementType::VACUUM_STATEMENT: return DUCKDB_STATEMENT_TYPE_VACUUM; case duckdb::StatementType::CALL_STATEMENT: diff --git a/src/duckdb/src/main/capi/hugeint-c.cpp b/src/duckdb/src/main/capi/hugeint-c.cpp index 337ec498c..bafed1efd 100644 --- a/src/duckdb/src/main/capi/hugeint-c.cpp +++ b/src/duckdb/src/main/capi/hugeint-c.cpp @@ -1,5 +1,6 @@ #include "duckdb/main/capi/capi_internal.hpp" #include "duckdb/common/types/hugeint.hpp" +#include "duckdb/common/types/uhugeint.hpp" #include "duckdb/common/types/decimal.hpp" #include "duckdb/common/operator/decimal_cast_operators.hpp" #include "duckdb/main/capi/cast/utils.hpp" @@ -7,6 +8,8 @@ using duckdb::Hugeint; using duckdb::hugeint_t; +using duckdb::Uhugeint; +using duckdb::uhugeint_t; using duckdb::Value; double duckdb_hugeint_to_double(duckdb_hugeint val) { @@ -16,6 +19,13 @@ double duckdb_hugeint_to_double(duckdb_hugeint val) { return Hugeint::Cast(internal); } +double duckdb_uhugeint_to_double(duckdb_uhugeint val) { + uhugeint_t internal; + internal.lower = val.lower; + internal.upper = val.upper; + return Uhugeint::Cast(internal); +} + static duckdb_decimal to_decimal_cast(double val, uint8_t width, uint8_t scale) { if (width > duckdb::Decimal::MAX_WIDTH_INT64) { return duckdb::TryCastToDecimalCInternal>(val, width, scale); @@ -49,6 +59,19 @@ duckdb_hugeint duckdb_double_to_hugeint(double val) { return result; } +duckdb_uhugeint duckdb_double_to_uhugeint(double val) { + uhugeint_t internal_result; + if (!Value::DoubleIsFinite(val) || !Uhugeint::TryConvert(val, internal_result)) { + internal_result.lower = 0; + internal_result.upper = 0; + } + + duckdb_uhugeint result; + result.lower = internal_result.lower; + result.upper = internal_result.upper; + return result; +} + double duckdb_decimal_to_double(duckdb_decimal val) { double result; hugeint_t value; diff --git a/src/duckdb/src/main/capi/logical_types-c.cpp b/src/duckdb/src/main/capi/logical_types-c.cpp index f4085ba3c..6a50fba7a 100644 --- a/src/duckdb/src/main/capi/logical_types-c.cpp +++ b/src/duckdb/src/main/capi/logical_types-c.cpp @@ -35,17 +35,17 @@ duckdb_logical_type duckdb_create_list_type(duckdb_logical_type type) { return reinterpret_cast(ltype); } -duckdb_logical_type duckdb_create_union_type(duckdb_logical_type member_types_p, const char **member_names, +duckdb_logical_type duckdb_create_union_type(duckdb_logical_type *member_types_p, const char **member_names, idx_t member_count) { if (!member_types_p || !member_names) { return nullptr; } - duckdb::LogicalType *member_types = reinterpret_cast(member_types_p); + duckdb::LogicalType **member_types = reinterpret_cast(member_types_p); duckdb::LogicalType *mtype = new duckdb::LogicalType; duckdb::child_list_t members; for (idx_t i = 0; i < member_count; i++) { - members.push_back(make_pair(member_names[i], member_types[i])); + members.push_back(make_pair(member_names[i], *member_types[i])); } *mtype = duckdb::LogicalType::UNION(members); return reinterpret_cast(mtype); diff --git a/src/duckdb/src/main/capi/pending-c.cpp b/src/duckdb/src/main/capi/pending-c.cpp index 300513900..734754ff2 100644 --- a/src/duckdb/src/main/capi/pending-c.cpp +++ b/src/duckdb/src/main/capi/pending-c.cpp @@ -1,7 +1,7 @@ #include "duckdb/main/capi/capi_internal.hpp" #include "duckdb/main/query_result.hpp" #include "duckdb/main/pending_query_result.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/case_insensitive_map.hpp" #include "duckdb/common/optional_ptr.hpp" @@ -25,10 +25,8 @@ duckdb_state duckdb_pending_prepared_internal(duckdb_prepared_statement prepared try { result->statement = wrapper->statement->PendingQuery(wrapper->values, allow_streaming); - } catch (const duckdb::Exception &ex) { - result->statement = make_uniq(duckdb::PreservedError(ex)); } catch (std::exception &ex) { - result->statement = make_uniq(duckdb::PreservedError(ex)); + result->statement = make_uniq(duckdb::ErrorData(ex)); } duckdb_state return_value = !result->statement->HasError() ? DuckDBSuccess : DuckDBError; *out_result = reinterpret_cast(result); @@ -82,11 +80,8 @@ duckdb_pending_state duckdb_pending_execute_task(duckdb_pending_result pending_r PendingExecutionResult return_value; try { return_value = wrapper->statement->ExecuteTask(); - } catch (const duckdb::Exception &ex) { - wrapper->statement->SetError(duckdb::PreservedError(ex)); - return DUCKDB_PENDING_ERROR; } catch (std::exception &ex) { - wrapper->statement->SetError(duckdb::PreservedError(ex)); + wrapper->statement->SetError(duckdb::ErrorData(ex)); return DUCKDB_PENDING_ERROR; } switch (return_value) { diff --git a/src/duckdb/src/main/capi/prepared-c.cpp b/src/duckdb/src/main/capi/prepared-c.cpp index e42742ca3..0569114ef 100644 --- a/src/duckdb/src/main/capi/prepared-c.cpp +++ b/src/duckdb/src/main/capi/prepared-c.cpp @@ -2,6 +2,7 @@ #include "duckdb/main/query_result.hpp" #include "duckdb/main/prepared_statement_data.hpp" #include "duckdb/common/types/decimal.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/common/optional_ptr.hpp" #include "duckdb/common/case_insensitive_map.hpp" @@ -9,6 +10,7 @@ using duckdb::case_insensitive_map_t; using duckdb::Connection; using duckdb::date_t; using duckdb::dtime_t; +using duckdb::ErrorData; using duckdb::ExtractStatementsWrapper; using duckdb::hugeint_t; using duckdb::LogicalType; @@ -18,6 +20,7 @@ using duckdb::PreparedStatementWrapper; using duckdb::QueryResultType; using duckdb::StringUtil; using duckdb::timestamp_t; +using duckdb::uhugeint_t; using duckdb::Value; idx_t duckdb_extract_statements(duckdb_connection connection, const char *query, @@ -29,8 +32,9 @@ idx_t duckdb_extract_statements(duckdb_connection connection, const char *query, Connection *conn = reinterpret_cast(connection); try { wrapper->statements = conn->ExtractStatements(query); - } catch (const duckdb::ParserException &e) { - wrapper->error = e.what(); + } catch (const std::exception &ex) { + ErrorData error(ex); + wrapper->error = error.Message(); } *out_extracted_statements = (duckdb_extracted_statements)wrapper; @@ -213,11 +217,23 @@ static hugeint_t duckdb_internal_hugeint(duckdb_hugeint val) { return internal; } +static uhugeint_t duckdb_internal_uhugeint(duckdb_uhugeint val) { + uhugeint_t internal; + internal.lower = val.lower; + internal.upper = val.upper; + return internal; +} + duckdb_state duckdb_bind_hugeint(duckdb_prepared_statement prepared_statement, idx_t param_idx, duckdb_hugeint val) { auto value = Value::HUGEINT(duckdb_internal_hugeint(val)); return duckdb_bind_value(prepared_statement, param_idx, (duckdb_value)&value); } +duckdb_state duckdb_bind_uhugeint(duckdb_prepared_statement prepared_statement, idx_t param_idx, duckdb_uhugeint val) { + auto value = Value::UHUGEINT(duckdb_internal_uhugeint(val)); + return duckdb_bind_value(prepared_statement, param_idx, (duckdb_value)&value); +} + duckdb_state duckdb_bind_uint8(duckdb_prepared_statement prepared_statement, idx_t param_idx, uint8_t val) { auto value = Value::UTINYINT(val); return duckdb_bind_value(prepared_statement, param_idx, (duckdb_value)&value); diff --git a/src/duckdb/src/main/capi/result-c.cpp b/src/duckdb/src/main/capi/result-c.cpp index d646500b4..5f4070725 100644 --- a/src/duckdb/src/main/capi/result-c.cpp +++ b/src/duckdb/src/main/capi/result-c.cpp @@ -82,6 +82,16 @@ struct CHugeintConverter : public CBaseConverter { } }; +struct CUhugeintConverter : public CBaseConverter { + template + static DST Convert(SRC input) { + duckdb_uhugeint result; + result.lower = input.lower; + result.upper = input.upper; + return result; + } +}; + struct CIntervalConverter : public CBaseConverter { template static DST Convert(SRC input) { @@ -215,6 +225,10 @@ duckdb_state deprecated_duckdb_translate_column(MaterializedQueryResult &result, WriteData(column, collection, column_ids); break; } + case LogicalTypeId::UHUGEINT: { + WriteData(column, collection, column_ids); + break; + } case LogicalTypeId::INTERVAL: { WriteData(column, collection, column_ids); break; diff --git a/src/duckdb/src/main/capi/table_function-c.cpp b/src/duckdb/src/main/capi/table_function-c.cpp index fe1556bfa..e6eb5e354 100644 --- a/src/duckdb/src/main/capi/table_function-c.cpp +++ b/src/duckdb/src/main/capi/table_function-c.cpp @@ -118,7 +118,7 @@ unique_ptr CTableFunctionBind(ClientContext &context, TableFunctio CTableInternalBindInfo bind_info(context, input, return_types, names, *result, info); info.bind(&bind_info); if (!bind_info.success) { - throw Exception(bind_info.error); + throw BinderException(bind_info.error); } return std::move(result); @@ -131,7 +131,7 @@ unique_ptr CTableFunctionInit(ClientContext &context, CTableInternalInitInfo init_info(bind_data, result->init_data, data_p.column_ids, data_p.filters); bind_data.info.init(&init_info); if (!init_info.success) { - throw Exception(init_info.error); + throw InvalidInputException(init_info.error); } return std::move(result); } @@ -147,7 +147,7 @@ unique_ptr CTableFunctionLocalInit(ExecutionContext &co CTableInternalInitInfo init_info(bind_data, result->init_data, data_p.column_ids, data_p.filters); bind_data.info.local_init(&init_info); if (!init_info.success) { - throw Exception(init_info.error); + throw InvalidInputException(init_info.error); } return std::move(result); } @@ -167,7 +167,7 @@ void CTableFunction(ClientContext &context, TableFunctionInput &data_p, DataChun CTableInternalFunctionInfo function_info(bind_data, global_data.init_data, local_data.init_data); bind_data.info.function(&function_info, reinterpret_cast(&output)); if (!function_info.success) { - throw Exception(function_info.error); + throw InvalidInputException(function_info.error); } } diff --git a/src/duckdb/src/main/capi/value-c.cpp b/src/duckdb/src/main/capi/value-c.cpp index a4a6040b9..931052614 100644 --- a/src/duckdb/src/main/capi/value-c.cpp +++ b/src/duckdb/src/main/capi/value-c.cpp @@ -3,6 +3,7 @@ #include "duckdb/common/types/time.hpp" #include "duckdb/common/types/timestamp.hpp" #include "duckdb/common/types.hpp" +#include "duckdb/common/uhugeint.hpp" #include "duckdb/main/capi/cast/generic.hpp" @@ -17,6 +18,7 @@ using duckdb::interval_t; using duckdb::StringCast; using duckdb::timestamp_t; using duckdb::ToCStringCastWrapper; +using duckdb::uhugeint_t; using duckdb::UnsafeFetch; bool duckdb_value_boolean(duckdb_result *result, idx_t col, idx_t row) { @@ -68,6 +70,14 @@ duckdb_hugeint duckdb_value_hugeint(duckdb_result *result, idx_t col, idx_t row) return result_value; } +duckdb_uhugeint duckdb_value_uhugeint(duckdb_result *result, idx_t col, idx_t row) { + duckdb_uhugeint result_value; + auto internal_value = GetInternalCValue(result, col, row); + result_value.lower = internal_value.lower; + result_value.upper = internal_value.upper; + return result_value; +} + uint8_t duckdb_value_uint8(duckdb_result *result, idx_t col, idx_t row) { return GetInternalCValue(result, col, row); } diff --git a/src/duckdb/src/main/chunk_scan_state/query_result.cpp b/src/duckdb/src/main/chunk_scan_state/query_result.cpp index 84e45e364..1c533f8c2 100644 --- a/src/duckdb/src/main/chunk_scan_state/query_result.cpp +++ b/src/duckdb/src/main/chunk_scan_state/query_result.cpp @@ -10,7 +10,7 @@ QueryResultChunkScanState::QueryResultChunkScanState(QueryResult &result) : Chun QueryResultChunkScanState::~QueryResultChunkScanState() { } -bool QueryResultChunkScanState::InternalLoad(PreservedError &error) { +bool QueryResultChunkScanState::InternalLoad(ErrorData &error) { D_ASSERT(!finished); if (result.type == QueryResultType::STREAM_RESULT) { auto &stream_result = result.Cast(); @@ -25,7 +25,7 @@ bool QueryResultChunkScanState::HasError() const { return result.HasError(); } -PreservedError &QueryResultChunkScanState::GetError() { +ErrorData &QueryResultChunkScanState::GetError() { D_ASSERT(result.HasError()); return result.GetErrorObject(); } @@ -38,7 +38,7 @@ const vector &QueryResultChunkScanState::Names() const { return result.names; } -bool QueryResultChunkScanState::LoadNextChunk(PreservedError &error) { +bool QueryResultChunkScanState::LoadNextChunk(ErrorData &error) { if (finished) { return !finished; } diff --git a/src/duckdb/src/main/client_context.cpp b/src/duckdb/src/main/client_context.cpp index 21ebd663a..e533dc158 100644 --- a/src/duckdb/src/main/client_context.cpp +++ b/src/duckdb/src/main/client_context.cpp @@ -5,7 +5,7 @@ #include "duckdb/catalog/catalog_search_path.hpp" #include "duckdb/common/file_system.hpp" #include "duckdb/common/http_state.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/progress_bar/progress_bar.hpp" #include "duckdb/common/serializer/buffered_file_writer.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" @@ -41,6 +41,8 @@ #include "duckdb/planner/pragma_handler.hpp" #include "duckdb/transaction/meta_transaction.hpp" #include "duckdb/transaction/transaction_manager.hpp" +#include "duckdb/storage/data_table.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -85,6 +87,20 @@ void ClientContext::Destroy() { CleanupInternal(*lock); } +void ClientContext::ProcessError(ErrorData &error, const string &query) const { + if (config.errors_as_json) { + error.ConvertErrorToJSON(); + } else if (!query.empty()) { + error.AddErrorLocation(query); + } +} + +template +unique_ptr ClientContext::ErrorResult(ErrorData error, const string &query) { + ProcessError(error, query); + return make_uniq(std::move(error)); +} + unique_ptr ClientContext::Fetch(ClientContextLock &lock, StreamQueryResult &result) { D_ASSERT(IsActiveResult(lock, &result)); D_ASSERT(active_query->executor); @@ -101,21 +117,21 @@ unique_ptr ClientContext::FetchInternal(ClientContextLock &lock, Exec CleanupInternal(lock, &result); } return chunk; - } catch (StandardException &ex) { - // standard exceptions do not invalidate the current transaction - result.SetError(PreservedError(ex)); - invalidate_query = false; - } catch (FatalException &ex) { - // fatal exceptions invalidate the entire database - result.SetError(PreservedError(ex)); - auto &db_inst = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_inst, ex.what()); - } catch (const Exception &ex) { - result.SetError(PreservedError(ex)); } catch (std::exception &ex) { - result.SetError(PreservedError(ex)); + ErrorData error(ex); + auto exception_type = error.Type(); + if (!Exception::InvalidatesTransaction(exception_type)) { + // standard exceptions do not invalidate the current transaction + invalidate_query = false; + } else if (Exception::InvalidatesDatabase(exception_type)) { + // fatal exceptions invalidate the entire database + auto &db_inst = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_inst, error.RawMessage()); + } + ProcessError(error, active_query->query); + result.SetError(std::move(error)); } catch (...) { // LCOV_EXCL_START - result.SetError(PreservedError("Unhandled exception in FetchInternal")); + result.SetError(ErrorData("Unhandled exception in FetchInternal")); } // LCOV_EXCL_STOP CleanupInternal(lock, &result, invalidate_query); return nullptr; @@ -126,12 +142,11 @@ void ClientContext::BeginTransactionInternal(ClientContextLock &lock, bool requi D_ASSERT(!active_query); auto &db_inst = DatabaseInstance::GetDatabase(*this); if (ValidChecker::IsInvalidated(db_inst)) { - throw FatalException(ErrorManager::FormatException(*this, ErrorType::INVALIDATED_DATABASE, - ValidChecker::InvalidatedMessage(db_inst))); + throw ErrorManager::InvalidatedDatabase(*this, ValidChecker::InvalidatedMessage(db_inst)); } if (requires_valid_transaction && transaction.HasActiveTransaction() && ValidChecker::IsInvalidated(transaction.ActiveTransaction())) { - throw Exception(ErrorManager::FormatException(*this, ErrorType::INVALIDATED_TRANSACTION)); + throw ErrorManager::InvalidatedTransaction(*this); } active_query = make_uniq(); if (transaction.IsAutoCommit()) { @@ -148,13 +163,9 @@ void ClientContext::BeginQueryInternal(ClientContextLock &lock, const string &qu transaction.SetActiveQuery(db->GetDatabaseManager().GetNewQueryNumber()); } -PreservedError ClientContext::EndQueryInternal(ClientContextLock &lock, bool success, bool invalidate_transaction) { +ErrorData ClientContext::EndQueryInternal(ClientContextLock &lock, bool success, bool invalidate_transaction) { client_data->profiler->EndQuery(); - if (client_data->http_state) { - client_data->http_state->Reset(); - } - // Notify any registered state of query end for (auto const &s : registered_state) { s.second->QueryEnd(); @@ -163,7 +174,7 @@ PreservedError ClientContext::EndQueryInternal(ClientContextLock &lock, bool suc D_ASSERT(active_query.get()); active_query.reset(); query_progress.Initialize(); - PreservedError error; + ErrorData error; try { if (transaction.HasActiveTransaction()) { // Move the query profiler into the history @@ -189,22 +200,19 @@ PreservedError ClientContext::EndQueryInternal(ClientContextLock &lock, bool suc ValidChecker::Invalidate(ActiveTransaction(), "Failed to commit"); } } - } catch (FatalException &ex) { - auto &db_inst = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_inst, ex.what()); - error = PreservedError(ex); - } catch (const Exception &ex) { - error = PreservedError(ex); } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); + if (Exception::InvalidatesDatabase(error.Type())) { + auto &db_inst = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_inst, error.RawMessage()); + } } catch (...) { // LCOV_EXCL_START - error = PreservedError("Unhandled exception!"); + error = ErrorData("Unhandled exception!"); } // LCOV_EXCL_STOP return error; } void ClientContext::CleanupInternal(ClientContextLock &lock, BaseQueryResult *result, bool invalidate_transaction) { - client_data->http_state = make_shared(); if (!active_query) { // no query currently active return; @@ -315,7 +323,6 @@ ClientContext::CreatePreparedStatement(ClientContextLock &lock, const string &qu } } - client_data->http_state = make_shared(); planner.CreatePlan(std::move(statement)); D_ASSERT(planner.plan || !planner.properties.bound_all_parameters); profiler.EndPhase(); @@ -369,7 +376,7 @@ unique_ptr ClientContext::PendingPreparedStatement(ClientCon D_ASSERT(active_query); auto &statement = *statement_p; if (ValidChecker::IsInvalidated(ActiveTransaction()) && statement.properties.requires_valid_transaction) { - throw Exception(ErrorManager::FormatException(*this, ErrorType::INVALIDATED_TRANSACTION)); + throw ErrorManager::InvalidatedTransaction(*this); } auto &meta_transaction = MetaTransaction::Get(*this); auto &manager = DatabaseManager::Get(*this); @@ -379,7 +386,7 @@ unique_ptr ClientContext::PendingPreparedStatement(ClientCon throw InternalException("Database \"%s\" not found", modified_database); } if (entry->IsReadOnly()) { - throw Exception(StringUtil::Format( + throw InvalidInputException(StringUtil::Format( "Cannot execute statement of type \"%s\" on database \"%s\" which is attached in read-only mode!", StatementTypeToString(statement.statement_type), modified_database)); } @@ -435,6 +442,7 @@ unique_ptr ClientContext::PendingPreparedStatement(ClientCon PendingExecutionResult ClientContext::ExecuteTaskInternal(ClientContextLock &lock, PendingQueryResult &result) { D_ASSERT(active_query); D_ASSERT(active_query->open_result == &result); + bool invalidate_transaction = true; try { auto query_result = active_query->executor->ExecuteTask(); if (active_query->progress_bar) { @@ -442,19 +450,20 @@ PendingExecutionResult ClientContext::ExecuteTaskInternal(ClientContextLock &loc query_progress = active_query->progress_bar->GetDetailedQueryProgress(); } return query_result; - } catch (FatalException &ex) { - // fatal exceptions invalidate the entire database - result.SetError(PreservedError(ex)); - auto &db_instance = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_instance, ex.what()); - } catch (const Exception &ex) { - result.SetError(PreservedError(ex)); } catch (std::exception &ex) { - result.SetError(PreservedError(ex)); + auto error = ErrorData(ex); + if (!Exception::InvalidatesTransaction(error.Type())) { + invalidate_transaction = false; + } else if (Exception::InvalidatesDatabase(error.Type())) { + // fatal exceptions invalidate the entire database + auto &db_instance = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_instance, error.RawMessage()); + } + result.SetError(std::move(error)); } catch (...) { // LCOV_EXCL_START - result.SetError(PreservedError("Unhandled exception in ExecuteTaskInternal")); + result.SetError(ErrorData("Unhandled exception in ExecuteTaskInternal")); } // LCOV_EXCL_STOP - EndQueryInternal(lock, false, true); + EndQueryInternal(lock, false, invalidate_transaction); return PendingExecutionResult::EXECUTION_ERROR; } @@ -491,11 +500,10 @@ unique_ptr ClientContext::ExtractPlan(const string &query) { auto statements = ParseStatementsInternal(*lock, query); if (statements.size() != 1) { - throw Exception("ExtractPlan can only prepare a single statement"); + throw InvalidInputException("ExtractPlan can only prepare a single statement"); } unique_ptr plan; - client_data->http_state = make_shared(); RunFunctionInTransactionInternal(*lock, [&]() { Planner planner(*this); planner.CreatePlan(std::move(statements[0])); @@ -534,13 +542,12 @@ unique_ptr ClientContext::PrepareInternal(ClientContextLock & unique_ptr ClientContext::Prepare(unique_ptr statement) { auto lock = LockContext(); // prepare the query + auto query = statement->query; try { InitialCleanup(*lock); return PrepareInternal(*lock, std::move(statement)); - } catch (const Exception &ex) { - return make_uniq(PreservedError(ex)); } catch (std::exception &ex) { - return make_uniq(PreservedError(ex)); + return ErrorResult(ErrorData(ex), query); } } @@ -553,16 +560,14 @@ unique_ptr ClientContext::Prepare(const string &query) { // first parse the query auto statements = ParseStatementsInternal(*lock, query); if (statements.empty()) { - throw Exception("No statement to prepare!"); + throw InvalidInputException("No statement to prepare!"); } if (statements.size() > 1) { - throw Exception("Cannot prepare multiple statements at once!"); + throw InvalidInputException("Cannot prepare multiple statements at once!"); } return PrepareInternal(*lock, std::move(statements[0])); - } catch (const Exception &ex) { - return make_uniq(PreservedError(ex)); } catch (std::exception &ex) { - return make_uniq(PreservedError(ex)); + return ErrorResult(ErrorData(ex), query); } } @@ -571,10 +576,8 @@ unique_ptr ClientContext::PendingQueryPreparedInternal(Clien const PendingQueryParameters ¶meters) { try { InitialCleanup(lock); - } catch (const Exception &ex) { - return make_uniq(PreservedError(ex)); } catch (std::exception &ex) { - return make_uniq(PreservedError(ex)); + return ErrorResult(ErrorData(ex), query); } return PendingStatementOrPreparedStatementInternal(lock, query, nullptr, prepared, parameters); } @@ -591,7 +594,7 @@ unique_ptr ClientContext::Execute(const string &query, shared_ptr

HasError()) { - return make_uniq(pending->GetErrorObject()); + return ErrorResult(pending->GetErrorObject()); } return pending->ExecuteInternal(*lock); } @@ -613,10 +616,10 @@ unique_ptr ClientContext::PendingStatementInternal(ClientCon if (prepared->properties.parameter_count > 0 && parameter_count == 0) { string error_message = StringUtil::Format("Expected %lld parameters, but none were supplied", prepared->properties.parameter_count); - return make_uniq(PreservedError(error_message)); + return ErrorResult(ErrorData(error_message), query); } if (!prepared->properties.bound_all_parameters) { - return make_uniq(PreservedError("Not all parameters were bound")); + return ErrorResult(ErrorData("Not all parameters were bound"), query); } // execute the prepared statement return PendingPreparedStatement(lock, std::move(prepared), parameters); @@ -629,7 +632,7 @@ unique_ptr ClientContext::RunStatementInternal(ClientContextLock &l parameters.allow_stream_result = allow_stream_result; auto pending = PendingQueryInternal(lock, std::move(statement), parameters, verify); if (pending->HasError()) { - return make_uniq(pending->GetErrorObject()); + return ErrorResult(pending->GetErrorObject()); } return ExecutePendingQueryInternal(lock, *pending); } @@ -653,17 +656,15 @@ unique_ptr ClientContext::PendingStatementOrPreparedStatemen switch (statement->type) { case StatementType::SELECT_STATEMENT: { // in case this is a select query, we verify the original statement - PreservedError error; + ErrorData error; try { error = VerifyQuery(lock, query, std::move(statement)); - } catch (const Exception &ex) { - error = PreservedError(ex); } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); } - if (error) { + if (error.HasError()) { // error in verifying query - return make_uniq(error); + return ErrorResult(std::move(error), query); } statement = std::move(copied_statement); break; @@ -674,17 +675,15 @@ unique_ptr ClientContext::PendingStatementOrPreparedStatemen case StatementType::DELETE_STATEMENT: case StatementType::UPDATE_STATEMENT: { Parser parser; - PreservedError error; + ErrorData error; try { parser.ParseQuery(statement->ToString()); - } catch (const Exception &ex) { - error = PreservedError(ex); } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); } - if (error) { + if (error.HasError()) { // error in verifying query - return make_uniq(error); + return ErrorResult(std::move(error), query); } statement = std::move(parser.statements[0]); break; @@ -705,16 +704,14 @@ unique_ptr ClientContext::PendingStatementOrPreparedStatemen try { BeginQueryInternal(lock, query); - } catch (FatalException &ex) { - // fatal exceptions invalidate the entire database - auto &db_instance = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_instance, ex.what()); - result = make_uniq(PreservedError(ex)); - return result; - } catch (const Exception &ex) { - return make_uniq(PreservedError(ex)); } catch (std::exception &ex) { - return make_uniq(PreservedError(ex)); + ErrorData error(ex); + if (Exception::InvalidatesDatabase(error.Type())) { + // fatal exceptions invalidate the entire database + auto &db_instance = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_instance, error.RawMessage()); + } + return ErrorResult(std::move(error), query); } // start the profiler auto &profiler = QueryProfiler::Get(*this); @@ -736,23 +733,20 @@ unique_ptr ClientContext::PendingStatementOrPreparedStatemen } result = PendingPreparedStatement(lock, prepared, parameters); } - } catch (StandardException &ex) { - // standard exceptions do not invalidate the current transaction - result = make_uniq(PreservedError(ex)); - invalidate_query = false; - } catch (FatalException &ex) { - // fatal exceptions invalidate the entire database - if (!config.query_verification_enabled) { - auto &db_instance = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_instance, ex.what()); - } - result = make_uniq(PreservedError(ex)); - } catch (const Exception &ex) { - // other types of exceptions do invalidate the current transaction - result = make_uniq(PreservedError(ex)); } catch (std::exception &ex) { + ErrorData error(ex); + if (!Exception::InvalidatesTransaction(error.Type())) { + // standard exceptions do not invalidate the current transaction + invalidate_query = false; + } else if (Exception::InvalidatesDatabase(error.Type())) { + // fatal exceptions invalidate the entire database + if (!config.query_verification_enabled) { + auto &db_instance = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_instance, error.RawMessage()); + } + } // other types of exceptions do invalidate the current transaction - result = make_uniq(PreservedError(ex)); + result = ErrorResult(std::move(error), query); } if (result->HasError()) { // query failed: abort now @@ -788,7 +782,7 @@ void ClientContext::LogQueryInternal(ClientContextLock &, const string &query) { unique_ptr ClientContext::Query(unique_ptr statement, bool allow_stream_result) { auto pending_query = PendingQuery(std::move(statement), allow_stream_result); if (pending_query->HasError()) { - return make_uniq(pending_query->GetErrorObject()); + return ErrorResult(pending_query->GetErrorObject()); } return pending_query->Execute(); } @@ -796,10 +790,10 @@ unique_ptr ClientContext::Query(unique_ptr statement, unique_ptr ClientContext::Query(const string &query, bool allow_stream_result) { auto lock = LockContext(); - PreservedError error; + ErrorData error; vector> statements; if (!ParseStatements(*lock, query, statements, error)) { - return make_uniq(std::move(error)); + return ErrorResult(std::move(error), query); } if (statements.empty()) { // no statements, return empty successful result @@ -822,7 +816,7 @@ unique_ptr ClientContext::Query(const string &query, bool allow_str auto has_result = pending_query->properties.return_type == StatementReturnType::QUERY_RESULT; unique_ptr current_result; if (pending_query->HasError()) { - current_result = make_uniq(pending_query->GetErrorObject()); + current_result = ErrorResult(pending_query->GetErrorObject()); } else { current_result = ExecutePendingQueryInternal(*lock, *pending_query); } @@ -846,17 +840,14 @@ unique_ptr ClientContext::Query(const string &query, bool allow_str } bool ClientContext::ParseStatements(ClientContextLock &lock, const string &query, - vector> &result, PreservedError &error) { + vector> &result, ErrorData &error) { try { InitialCleanup(lock); // parse the query and transform it into a set of statements result = ParseStatementsInternal(lock, query); return true; - } catch (const Exception &ex) { - error = PreservedError(ex); - return false; } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); return false; } } @@ -864,13 +855,13 @@ bool ClientContext::ParseStatements(ClientContextLock &lock, const string &query unique_ptr ClientContext::PendingQuery(const string &query, bool allow_stream_result) { auto lock = LockContext(); - PreservedError error; + ErrorData error; vector> statements; if (!ParseStatements(*lock, query, statements, error)) { - return make_uniq(std::move(error)); + return ErrorResult(std::move(error), query); } if (statements.size() != 1) { - return make_uniq(PreservedError("PendingQuery can only take a single statement")); + return ErrorResult(ErrorData("PendingQuery can only take a single statement"), query); } PendingQueryParameters parameters; parameters.allow_stream_result = allow_stream_result; @@ -880,6 +871,13 @@ unique_ptr ClientContext::PendingQuery(const string &query, unique_ptr ClientContext::PendingQuery(unique_ptr statement, bool allow_stream_result) { auto lock = LockContext(); + + try { + InitialCleanup(*lock); + } catch (std::exception &ex) { + return ErrorResult(ErrorData(ex)); + } + PendingQueryParameters parameters; parameters.allow_stream_result = allow_stream_result; return PendingQueryInternal(*lock, std::move(statement), parameters); @@ -950,20 +948,20 @@ void ClientContext::RunFunctionInTransactionInternal(ClientContextLock &lock, co } try { fun(); - } catch (StandardException &ex) { - if (require_new_transaction) { - transaction.Rollback(); - } - throw; - } catch (FatalException &ex) { - auto &db_instance = DatabaseInstance::GetDatabase(*this); - ValidChecker::Invalidate(db_instance, ex.what()); - throw; } catch (std::exception &ex) { + ErrorData error(ex); + bool invalidates_transaction = true; + if (!Exception::InvalidatesTransaction(error.Type())) { + // standard exceptions don't invalidate the transaction + invalidates_transaction = false; + } else if (Exception::InvalidatesDatabase(error.Type())) { + auto &db_instance = DatabaseInstance::GetDatabase(*this); + ValidChecker::Invalidate(db_instance, error.RawMessage()); + } if (require_new_transaction) { transaction.Rollback(); - } else { - ValidChecker::Invalidate(ActiveTransaction(), ex.what()); + } else if (invalidates_transaction) { + ValidChecker::Invalidate(ActiveTransaction(), error.RawMessage()); } throw; } @@ -1003,11 +1001,11 @@ void ClientContext::Append(TableDescription &description, ColumnDataCollection & Catalog::GetEntry(*this, INVALID_CATALOG, description.schema, description.table); // verify that the table columns and types match up if (description.columns.size() != table_entry.GetColumns().PhysicalColumnCount()) { - throw Exception("Failed to append: table entry has different number of columns!"); + throw InvalidInputException("Failed to append: table entry has different number of columns!"); } for (idx_t i = 0; i < description.columns.size(); i++) { if (description.columns[i].Type() != table_entry.GetColumns().GetColumn(PhysicalIndex(i)).Type()) { - throw Exception("Failed to append: table entry has different number of columns!"); + throw InvalidInputException("Failed to append: table entry has different number of columns!"); } } table_entry.GetStorage().LocalAppend(table_entry, *this, collection); @@ -1019,7 +1017,6 @@ void ClientContext::TryBindRelation(Relation &relation, vector D_ASSERT(!relation.GetAlias().empty()); D_ASSERT(!relation.ToString().empty()); #endif - client_data->http_state = make_shared(); RunFunctionInTransaction([&]() { // bind the expressions auto binder = Binder::CreateBinder(*this); @@ -1087,7 +1084,7 @@ unique_ptr ClientContext::Execute(const shared_ptr &relat auto &expected_columns = relation->Columns(); auto pending = PendingQueryInternal(*lock, relation, false); if (!pending->success) { - return make_uniq(pending->GetErrorObject()); + return ErrorResult(pending->GetErrorObject()); } unique_ptr result; @@ -1123,7 +1120,7 @@ unique_ptr ClientContext::Execute(const shared_ptr &relat err_str += result->names[i] + " " + result->types[i].ToString(); } err_str += "]"; - return make_uniq(PreservedError(err_str)); + return ErrorResult(ErrorData(err_str)); } bool ClientContext::TryGetCurrentSetting(const std::string &key, Value &result) { diff --git a/src/duckdb/src/main/client_verify.cpp b/src/duckdb/src/main/client_verify.cpp index 8244eb798..f31a6fc52 100644 --- a/src/duckdb/src/main/client_verify.cpp +++ b/src/duckdb/src/main/client_verify.cpp @@ -1,4 +1,4 @@ -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/parser/statement/explain_statement.hpp" #include "duckdb/verification/statement_verifier.hpp" @@ -21,11 +21,16 @@ static void ThrowIfExceptionIsInternal(StatementVerifier &verifier) { } } -PreservedError ClientContext::VerifyQuery(ClientContextLock &lock, const string &query, - unique_ptr statement) { +ErrorData ClientContext::VerifyQuery(ClientContextLock &lock, const string &query, unique_ptr statement) { D_ASSERT(statement->type == StatementType::SELECT_STATEMENT); // Aggressive query verification +#ifdef DUCKDB_RUN_SLOW_VERIFIERS + bool run_slow_verifiers = true; +#else + bool run_slow_verifiers = false; +#endif + // The purpose of this function is to test correctness of otherwise hard to test features: // Copy() of statements and expressions // Serialize()/Deserialize() of expressions @@ -37,16 +42,28 @@ PreservedError ClientContext::VerifyQuery(ClientContextLock &lock, const string const auto &stmt = *statement; vector> statement_verifiers; unique_ptr prepared_statement_verifier; + + // Base Statement verifiers: these are the verifiers we enable for regular builds if (config.query_verification_enabled) { statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::COPIED, stmt)); statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::DESERIALIZED, stmt)); statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::UNOPTIMIZED, stmt)); prepared_statement_verifier = StatementVerifier::Create(VerificationType::PREPARED, stmt); + } + + // This verifier is enabled explicitly OR by enabling run_slow_verifiers + if (config.verify_fetch_row || (run_slow_verifiers && config.query_verification_enabled)) { + statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::FETCH_ROW_AS_SCAN, stmt)); + } + + // For the DEBUG_ASYNC build we enable this extra verifier #ifdef DUCKDB_DEBUG_ASYNC_SINK_SOURCE - // This verification is quite slow, so we only run it for the async sink/source debug mode + if (config.query_verification_enabled) { statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::NO_OPERATOR_CACHING, stmt)); -#endif } +#endif + + // Verify external always needs to be explicitly enabled and is never part of default verifier set if (config.verify_external) { statement_verifiers.emplace_back(StatementVerifier::Create(VerificationType::EXTERNAL, stmt)); } @@ -115,8 +132,9 @@ PreservedError ClientContext::VerifyQuery(ClientContextLock &lock, const string try { RunStatementInternal(lock, explain_q, std::move(explain_stmt), false, false); } catch (std::exception &ex) { // LCOV_EXCL_START + ErrorData error(ex); interrupted = false; - return PreservedError("EXPLAIN failed but query did not (" + string(ex.what()) + ")"); + return ErrorData("EXPLAIN failed but query did not (" + error.RawMessage() + ")"); } // LCOV_EXCL_STOP #ifdef DUCKDB_VERIFY_BOX_RENDERER @@ -142,11 +160,11 @@ PreservedError ClientContext::VerifyQuery(ClientContextLock &lock, const string for (auto &verifier : statement_verifiers) { auto result = original->CompareResults(*verifier); if (!result.empty()) { - return PreservedError(result); + return ErrorData(result); } } - return PreservedError(); + return ErrorData(); } } // namespace duckdb diff --git a/src/duckdb/src/main/config.cpp b/src/duckdb/src/main/config.cpp index 66fd43383..1c866eb73 100644 --- a/src/duckdb/src/main/config.cpp +++ b/src/duckdb/src/main/config.cpp @@ -54,6 +54,7 @@ bool DBConfigOptions::debug_print_bindings = false; { nullptr, nullptr, LogicalTypeId::INVALID, nullptr, nullptr, nullptr, nullptr, nullptr } static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting), + DUCKDB_GLOBAL(AllowPersistentSecrets), DUCKDB_GLOBAL(CheckpointThresholdSetting), DUCKDB_GLOBAL(DebugCheckpointAbort), DUCKDB_LOCAL(DebugForceExternal), @@ -69,8 +70,8 @@ static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting DUCKDB_GLOBAL(EnableExternalAccessSetting), DUCKDB_GLOBAL(EnableFSSTVectors), DUCKDB_GLOBAL(AllowUnsignedExtensionsSetting), - DUCKDB_LOCAL(CustomExtensionRepository), - DUCKDB_LOCAL(AutoloadExtensionRepository), + DUCKDB_GLOBAL(CustomExtensionRepository), + DUCKDB_GLOBAL(AutoloadExtensionRepository), DUCKDB_GLOBAL(AutoinstallKnownExtensions), DUCKDB_GLOBAL(AutoloadKnownExtensions), DUCKDB_GLOBAL(EnableObjectCacheSetting), @@ -78,6 +79,7 @@ static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting DUCKDB_LOCAL(EnableProfilingSetting), DUCKDB_LOCAL(EnableProgressBarSetting), DUCKDB_LOCAL(EnableProgressBarPrintSetting), + DUCKDB_LOCAL(ErrorsAsJsonSetting), DUCKDB_LOCAL(ExplainOutputSetting), DUCKDB_GLOBAL(ExtensionDirectorySetting), DUCKDB_GLOBAL(ExternalThreadsSetting), @@ -91,6 +93,7 @@ static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting DUCKDB_LOCAL(IntegerDivisionSetting), DUCKDB_LOCAL(MaximumExpressionDepthSetting), DUCKDB_GLOBAL(MaximumMemorySetting), + DUCKDB_GLOBAL(OldImplicitCasting), DUCKDB_GLOBAL_ALIAS("memory_limit", MaximumMemorySetting), DUCKDB_GLOBAL_ALIAS("null_order", DefaultNullOrderSetting), DUCKDB_LOCAL(OrderedAggregateThreshold), @@ -107,6 +110,8 @@ static ConfigurationOption internal_options[] = {DUCKDB_GLOBAL(AccessModeSetting DUCKDB_LOCAL(ProgressBarTimeSetting), DUCKDB_LOCAL(SchemaSetting), DUCKDB_LOCAL(SearchPathSetting), + DUCKDB_GLOBAL(SecretDirectorySetting), + DUCKDB_GLOBAL(DefaultSecretStorage), DUCKDB_GLOBAL(TempDirectorySetting), DUCKDB_GLOBAL(ThreadsSetting), DUCKDB_GLOBAL(UsernameSetting), @@ -243,6 +248,10 @@ CastFunctionSet &DBConfig::GetCastFunctions() { return *cast_functions; } +IndexTypeSet &DBConfig::GetIndexTypes() { + return *index_types; +} + void DBConfig::SetDefaultMaxMemory() { auto memory = FileSystem::GetAvailableMemory(); if (memory != DConstants::INVALID_INDEX) { @@ -438,7 +447,11 @@ OrderByNullType DBConfig::ResolveNullOrder(OrderType order_type, OrderByNullType } const std::string DBConfig::UserAgent() const { - auto user_agent = options.duckdb_api; + auto user_agent = GetDefaultUserAgent(); + + if (!options.duckdb_api.empty()) { + user_agent += " " + options.duckdb_api; + } if (!options.custom_user_agent.empty()) { user_agent += " " + options.custom_user_agent; diff --git a/src/duckdb/src/main/connection.cpp b/src/duckdb/src/main/connection.cpp index b1263bb1e..432ca9c21 100644 --- a/src/duckdb/src/main/connection.cpp +++ b/src/duckdb/src/main/connection.cpp @@ -1,7 +1,6 @@ #include "duckdb/main/connection.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" -#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/main/appender.hpp" #include "duckdb/main/client_context.hpp" diff --git a/src/duckdb/src/main/connection_manager.cpp b/src/duckdb/src/main/connection_manager.cpp index c8ca304ab..14e7f23a2 100644 --- a/src/duckdb/src/main/connection_manager.cpp +++ b/src/duckdb/src/main/connection_manager.cpp @@ -1,4 +1,5 @@ #include "duckdb/main/connection_manager.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/main/database.cpp b/src/duckdb/src/main/database.cpp index 7b5744068..d62f9f796 100644 --- a/src/duckdb/src/main/database.cpp +++ b/src/duckdb/src/main/database.cpp @@ -9,17 +9,19 @@ #include "duckdb/main/client_context.hpp" #include "duckdb/main/connection_manager.hpp" #include "duckdb/main/database_manager.hpp" +#include "duckdb/main/database_path_and_type.hpp" #include "duckdb/main/error_manager.hpp" #include "duckdb/main/extension_helper.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/parser/parsed_data/attach_info.hpp" +#include "duckdb/planner/extension_callback.hpp" #include "duckdb/storage/object_cache.hpp" #include "duckdb/storage/standard_buffer_manager.hpp" -#include "duckdb/main/database_path_and_type.hpp" #include "duckdb/storage/storage_extension.hpp" #include "duckdb/storage/storage_manager.hpp" #include "duckdb/transaction/transaction_manager.hpp" -#include "duckdb/planner/extension_callback.hpp" +#include "duckdb/execution/index/index_type_set.hpp" #ifndef DUCKDB_NO_THREADS #include "duckdb/common/thread.hpp" @@ -29,9 +31,9 @@ namespace duckdb { DBConfig::DBConfig() { compression_functions = make_uniq(); - cast_functions = make_uniq(); + cast_functions = make_uniq(*this); + index_types = make_uniq(); error_manager = make_uniq(); - options.duckdb_api = StringUtil::Format("duckdb/%s(%s)", DuckDB::LibraryVersion(), DuckDB::Platform()); } DBConfig::DBConfig(bool read_only) : DBConfig::DBConfig() { @@ -182,6 +184,10 @@ void DatabaseInstance::Initialize(const char *database_path, DBConfig *user_conf config_ptr = user_config; } + if (config_ptr->options.duckdb_api.empty()) { + config_ptr->SetOptionByName("duckdb_api", "cpp"); + } + if (config_ptr->options.temporary_directory.empty() && database_path) { // no directory specified: use default temp path config_ptr->options.temporary_directory = string(database_path) + ".tmp"; @@ -197,6 +203,7 @@ void DatabaseInstance::Initialize(const char *database_path, DBConfig *user_conf } else { config_ptr->options.database_path.clear(); } + Configure(*config_ptr); if (user_config && !user_config->options.use_temporary_directory) { @@ -213,6 +220,9 @@ void DatabaseInstance::Initialize(const char *database_path, DBConfig *user_conf // resolve the type of teh database we are opening DBPathAndType::ResolveDatabaseType(config.options.database_path, config.options.database_type, config); + // initialize the secret manager + config.secret_manager->Initialize(*this); + // initialize the system catalog db_manager->InitializeSystemCatalog(); @@ -221,7 +231,7 @@ void DatabaseInstance::Initialize(const char *database_path, DBConfig *user_conf if (!config.file_system) { throw InternalException("No file system!?"); } - ExtensionHelper::LoadExternalExtension(*this, *config.file_system, config.options.database_type, nullptr); + ExtensionHelper::LoadExternalExtension(*this, *config.file_system, config.options.database_type); } if (!config.options.unrecognized_options.empty()) { @@ -252,11 +262,15 @@ DuckDB::DuckDB(DatabaseInstance &instance_p) : instance(instance_p.shared_from_t DuckDB::~DuckDB() { } +SecretManager &DatabaseInstance::GetSecretManager() { + return *config.secret_manager; +} + BufferManager &DatabaseInstance::GetBufferManager() { return *buffer_manager; } -BufferPool &DatabaseInstance::GetBufferPool() { +BufferPool &DatabaseInstance::GetBufferPool() const { return *config.buffer_pool; } @@ -311,6 +325,11 @@ void DatabaseInstance::Configure(DBConfig &new_config) { } else { config.file_system = make_uniq(); } + if (new_config.secret_manager) { + config.secret_manager = std::move(new_config.secret_manager); + } else { + config.secret_manager = make_uniq(); + } if (config.options.maximum_memory == (idx_t)-1) { config.SetDefaultMaxMemory(); } diff --git a/src/duckdb/src/main/error_manager.cpp b/src/duckdb/src/main/error_manager.cpp index 140381983..ece4ce439 100644 --- a/src/duckdb/src/main/error_manager.cpp +++ b/src/duckdb/src/main/error_manager.cpp @@ -1,6 +1,7 @@ #include "duckdb/main/error_manager.hpp" #include "duckdb/main/config.hpp" #include "utf8proc_wrapper.hpp" +#include "duckdb/common/exception/list.hpp" namespace duckdb { @@ -34,12 +35,12 @@ string ErrorManager::FormatExceptionRecursive(ErrorType error_type, vectorExtensionIsLoaded(extension_name)) { + return ExtensionHelper::AutoLoadExtension(*context.db, extension_name); +} + +void ExtensionHelper::AutoLoadExtension(DatabaseInstance &db, const string &extension_name) { + if (db.ExtensionIsLoaded(extension_name)) { // Avoid downloading again return; } - auto &dbconfig = DBConfig::GetConfig(context); + auto &dbconfig = DBConfig::GetConfig(db); try { + auto fs = FileSystem::CreateLocal(); #ifndef DUCKDB_WASM if (dbconfig.options.autoinstall_known_extensions) { - ExtensionHelper::InstallExtension(context, extension_name, false, - context.config.autoinstall_extension_repo); + ExtensionHelper::InstallExtension(db.config, *fs, extension_name, false, + dbconfig.options.autoinstall_extension_repo); } #endif - ExtensionHelper::LoadExternalExtension(context, extension_name); - } catch (Exception &e) { - throw AutoloadException(extension_name, e); + ExtensionHelper::LoadExternalExtension(db, *fs, extension_name); + } catch (std::exception &e) { + ErrorData error(e); + throw AutoloadException(extension_name, error.RawMessage()); } } @@ -341,13 +347,6 @@ ExtensionLoadResult ExtensionHelper::LoadExtensionInternal(DuckDB &db, const std db.LoadExtension(); #else return ExtensionLoadResult::NOT_LOADED; -#endif - } else if (extension == "visualizer") { -#if DUCKDB_EXTENSION_VISUALIZER_LINKED - db.LoadExtension(); -#else - // visualizer extension required but not build: skip this test - return ExtensionLoadResult::NOT_LOADED; #endif } else if (extension == "json") { #if DUCKDB_EXTENSION_JSON_LINKED diff --git a/src/duckdb/src/main/extension/extension_install.cpp b/src/duckdb/src/main/extension/extension_install.cpp index 90be2577c..418a298db 100644 --- a/src/duckdb/src/main/extension/extension_install.cpp +++ b/src/duckdb/src/main/extension/extension_install.cpp @@ -1,7 +1,8 @@ -#include "duckdb/main/extension_helper.hpp" #include "duckdb/common/gzip_file_system.hpp" #include "duckdb/common/types/uuid.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/common/exception/http_exception.hpp" +#include "duckdb/main/extension_helper.hpp" #ifndef DISABLE_DUCKDB_REMOTE_INSTALL #ifndef DUCKDB_DISABLE_EXTENSION_LOAD @@ -40,7 +41,21 @@ const string ExtensionHelper::GetVersionDirectoryName() { } const vector ExtensionHelper::PathComponents() { - return vector {".duckdb", "extensions", GetVersionDirectoryName(), DuckDB::Platform()}; + return vector {GetVersionDirectoryName(), DuckDB::Platform()}; +} + +duckdb::string ExtensionHelper::DefaultExtensionFolder(FileSystem &fs) { + string home_directory = fs.GetHomeDirectory(); + // exception if the home directory does not exist, don't create whatever we think is home + if (!fs.DirectoryExists(home_directory)) { + throw IOException("Can't find the home directory at '%s'\nSpecify a home directory using the SET " + "home_directory='/path/to/dir' option.", + home_directory); + } + string res = home_directory; + res = fs.JoinPath(res, ".duckdb"); + res = fs.JoinPath(res, "extensions"); + return res; } string ExtensionHelper::ExtensionDirectory(DBConfig &config, FileSystem &fs) { @@ -52,6 +67,10 @@ string ExtensionHelper::ExtensionDirectory(DBConfig &config, FileSystem &fs) { extension_directory = config.options.extension_directory; // TODO this should probably live in the FileSystem // convert random separators to platform-canonic + } else { // otherwise default to home + extension_directory = DefaultExtensionFolder(fs); + } + { extension_directory = fs.ConvertSeparators(extension_directory); // expand ~ in extension directory extension_directory = fs.ExpandPath(extension_directory); @@ -70,15 +89,6 @@ string ExtensionHelper::ExtensionDirectory(DBConfig &config, FileSystem &fs) { } } } - } else { // otherwise default to home - string home_directory = fs.GetHomeDirectory(); - // exception if the home directory does not exist, don't create whatever we think is home - if (!fs.DirectoryExists(home_directory)) { - throw IOException("Can't find the home directory at '%s'\nSpecify a home directory using the SET " - "home_directory='/path/to/dir' option.", - home_directory); - } - extension_directory = home_directory; } D_ASSERT(fs.DirectoryExists(extension_directory)); @@ -99,6 +109,7 @@ string ExtensionHelper::ExtensionDirectory(ClientContext &context) { } bool ExtensionHelper::CreateSuggestions(const string &extension_name, string &message) { + auto lowercase_extension_name = StringUtil::Lower(extension_name); vector candidates; for (idx_t ext_count = ExtensionHelper::DefaultExtensionCount(), i = 0; i < ext_count; i++) { candidates.emplace_back(ExtensionHelper::GetDefaultExtension(i).name); @@ -106,10 +117,10 @@ bool ExtensionHelper::CreateSuggestions(const string &extension_name, string &me for (idx_t ext_count = ExtensionHelper::ExtensionAliasCount(), i = 0; i < ext_count; i++) { candidates.emplace_back(ExtensionHelper::GetExtensionAlias(i).alias); } - auto closest_extensions = StringUtil::TopNLevenshtein(candidates, extension_name); + auto closest_extensions = StringUtil::TopNLevenshtein(candidates, lowercase_extension_name); message = StringUtil::CandidatesMessage(closest_extensions, "Candidate extensions"); for (auto &closest : closest_extensions) { - if (closest == extension_name) { + if (closest == lowercase_extension_name) { message = "Extension \"" + extension_name + "\" is an existing extension.\n"; return true; } @@ -124,7 +135,7 @@ void ExtensionHelper::InstallExtension(DBConfig &config, FileSystem &fs, const s return; #endif string local_path = ExtensionDirectory(config, fs); - InstallExtensionInternal(config, nullptr, fs, local_path, extension, force_install, repository); + InstallExtensionInternal(config, fs, local_path, extension, force_install, repository); } void ExtensionHelper::InstallExtension(ClientContext &context, const string &extension, bool force_install, @@ -136,8 +147,7 @@ void ExtensionHelper::InstallExtension(ClientContext &context, const string &ext auto &config = DBConfig::GetConfig(context); auto &fs = FileSystem::GetFileSystem(context); string local_path = ExtensionDirectory(context); - auto &client_config = ClientConfig::GetConfig(context); - InstallExtensionInternal(config, &client_config, fs, local_path, extension, force_install, repository); + InstallExtensionInternal(config, fs, local_path, extension, force_install, repository); } unsafe_unique_array ReadExtensionFileFromDisk(FileSystem &fs, const string &path, idx_t &file_size) { @@ -157,7 +167,7 @@ void WriteExtensionFileToDisk(FileSystem &fs, const string &path, void *data, id target_file.reset(); } -string ExtensionHelper::ExtensionUrlTemplate(optional_ptr client_config, const string &repository) { +string ExtensionHelper::ExtensionUrlTemplate(optional_ptr db_config, const string &repository) { string versioned_path = "/${REVISION}/${PLATFORM}/${NAME}.duckdb_extension"; #ifdef WASM_LOADABLE_EXTENSIONS string default_endpoint = "https://extensions.duckdb.org"; @@ -166,7 +176,7 @@ string ExtensionHelper::ExtensionUrlTemplate(optional_ptr cl string default_endpoint = "http://extensions.duckdb.org"; versioned_path = versioned_path + ".gz"; #endif - string custom_endpoint = client_config ? client_config->custom_extension_repo : string(); + string custom_endpoint = db_config ? db_config->options.custom_extension_repo : string(); string endpoint; if (!repository.empty()) { endpoint = repository; @@ -186,9 +196,8 @@ string ExtensionHelper::ExtensionFinalizeUrlTemplate(const string &url_template, return url; } -void ExtensionHelper::InstallExtensionInternal(DBConfig &config, ClientConfig *client_config, FileSystem &fs, - const string &local_path, const string &extension, bool force_install, - const string &repository) { +void ExtensionHelper::InstallExtensionInternal(DBConfig &config, FileSystem &fs, const string &local_path, + const string &extension, bool force_install, const string &repository) { #ifdef DUCKDB_DISABLE_EXTENSION_LOAD throw PermissionException("Installing external extensions is disabled through a compile time flag"); #else @@ -228,7 +237,7 @@ void ExtensionHelper::InstallExtensionInternal(DBConfig &config, ClientConfig *c throw BinderException("Remote extension installation is disabled through configuration"); #else - string url_template = ExtensionUrlTemplate(client_config, repository); + string url_template = ExtensionUrlTemplate(&config, repository); if (is_http_url) { url_template = extension; diff --git a/src/duckdb/src/main/extension/extension_load.cpp b/src/duckdb/src/main/extension/extension_load.cpp index 80d24c298..6cd662a8f 100644 --- a/src/duckdb/src/main/extension/extension_load.cpp +++ b/src/duckdb/src/main/extension/extension_load.cpp @@ -58,8 +58,7 @@ static void ComputeSHA256FileSegment(FileHandle *handle, const idx_t start, cons #endif bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const string &extension, - ExtensionInitResult &result, string &error, - optional_ptr client_config) { + ExtensionInitResult &result, string &error) { #ifdef DUCKDB_DISABLE_EXTENSION_LOAD throw PermissionException("Loading external extensions is disabled through a compile time flag"); #else @@ -72,7 +71,7 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str if (!ExtensionHelper::IsFullPath(extension)) { string extension_name = ApplyExtensionAlias(extension); #ifdef WASM_LOADABLE_EXTENSIONS - string url_template = ExtensionUrlTemplate(client_config, ""); + string url_template = ExtensionUrlTemplate(&config, ""); string url = ExtensionFinalizeUrlTemplate(url_template, extension_name); char *str = (char *)EM_ASM_PTR( @@ -95,8 +94,8 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str filename = address; #else - string local_path = - !config.options.extension_directory.empty() ? config.options.extension_directory : fs.GetHomeDirectory(); + string local_path = !config.options.extension_directory.empty() ? config.options.extension_directory + : ExtensionHelper::DefaultExtensionFolder(fs); // convert random separators to platform-canonic local_path = fs.ConvertSeparators(local_path); @@ -178,7 +177,7 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str throw IOException(config.error_manager->FormatException(ErrorType::UNSIGNED_EXTENSION, filename)); } } - auto basename = fs.ExtractBaseName(filename); + auto filebase = fs.ExtractBaseName(filename); #ifdef WASM_LOADABLE_EXTENSIONS EM_ASM( @@ -196,8 +195,8 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str // Here we add the uInt8Array to Emscripten's filesystem, for it to be found by dlopen FS.writeFile(UTF8ToString($1), new Uint8Array(uInt8Array)); }, - filename.c_str(), basename.c_str()); - auto dopen_from = basename; + filename.c_str(), filebase.c_str()); + auto dopen_from = filebase; #else auto dopen_from = filename; #endif @@ -207,8 +206,10 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str throw IOException("Extension \"%s\" could not be loaded: %s", filename, GetDLError()); } + auto lowercase_extension_name = StringUtil::Lower(filebase); + ext_version_fun_t version_fun; - auto version_fun_name = basename + "_version"; + auto version_fun_name = lowercase_extension_name + "_version"; version_fun = LoadFunctionFromDLL(lib_hdl, version_fun_name, filename); @@ -235,25 +236,24 @@ bool ExtensionHelper::TryInitialLoad(DBConfig &config, FileSystem &fs, const str extension_version, engine_version); } - result.basename = basename; + result.filebase = lowercase_extension_name; result.filename = filename; result.lib_hdl = lib_hdl; return true; #endif } -ExtensionInitResult ExtensionHelper::InitialLoad(DBConfig &config, FileSystem &fs, const string &extension, - optional_ptr client_config) { +ExtensionInitResult ExtensionHelper::InitialLoad(DBConfig &config, FileSystem &fs, const string &extension) { string error; ExtensionInitResult result; - if (!TryInitialLoad(config, fs, extension, result, error, client_config)) { + if (!TryInitialLoad(config, fs, extension, result, error)) { if (!ExtensionHelper::AllowAutoInstall(extension)) { throw IOException(error); } // the extension load failed - try installing the extension ExtensionHelper::InstallExtension(config, fs, extension, false); // try loading again - if (!TryInitialLoad(config, fs, extension, result, error, client_config)) { + if (!TryInitialLoad(config, fs, extension, result, error)) { throw IOException(error); } } @@ -281,16 +281,15 @@ string ExtensionHelper::GetExtensionName(const string &original_name) { return ExtensionHelper::ApplyExtensionAlias(splits.front()); } -void ExtensionHelper::LoadExternalExtension(DatabaseInstance &db, FileSystem &fs, const string &extension, - optional_ptr client_config) { +void ExtensionHelper::LoadExternalExtension(DatabaseInstance &db, FileSystem &fs, const string &extension) { if (db.ExtensionIsLoaded(extension)) { return; } #ifdef DUCKDB_DISABLE_EXTENSION_LOAD throw PermissionException("Loading external extensions is disabled through a compile time flag"); #else - auto res = InitialLoad(DBConfig::GetConfig(db), fs, extension, client_config); - auto init_fun_name = res.basename + "_init"; + auto res = InitialLoad(DBConfig::GetConfig(db), fs, extension); + auto init_fun_name = res.filebase + "_init"; ext_init_fun_t init_fun; init_fun = LoadFunctionFromDLL(res.lib_hdl, init_fun_name, res.filename); @@ -298,8 +297,9 @@ void ExtensionHelper::LoadExternalExtension(DatabaseInstance &db, FileSystem &fs try { (*init_fun)(db); } catch (std::exception &e) { + ErrorData error(e); throw InvalidInputException("Initialization function \"%s\" from file \"%s\" threw an exception: \"%s\"", - init_fun_name, res.filename, e.what()); + init_fun_name, res.filename, error.RawMessage()); } db.SetExtensionLoaded(extension); @@ -307,8 +307,7 @@ void ExtensionHelper::LoadExternalExtension(DatabaseInstance &db, FileSystem &fs } void ExtensionHelper::LoadExternalExtension(ClientContext &context, const string &extension) { - LoadExternalExtension(DatabaseInstance::GetDatabase(context), FileSystem::GetFileSystem(context), extension, - &ClientConfig::GetConfig(context)); + LoadExternalExtension(DatabaseInstance::GetDatabase(context), FileSystem::GetFileSystem(context), extension); } string ExtensionHelper::ExtractExtensionPrefixFromPath(const string &path) { diff --git a/src/duckdb/src/main/extension/extension_util.cpp b/src/duckdb/src/main/extension/extension_util.cpp index 383e8bbee..9d72104c4 100644 --- a/src/duckdb/src/main/extension/extension_util.cpp +++ b/src/duckdb/src/main/extension/extension_util.cpp @@ -12,6 +12,7 @@ #include "duckdb/parser/parsed_data/create_collation_info.hpp" #include "duckdb/catalog/catalog.hpp" #include "duckdb/main/config.hpp" +#include "duckdb/main/secret/secret_manager.hpp" namespace duckdb { @@ -45,6 +46,12 @@ void ExtensionUtil::RegisterFunction(DatabaseInstance &db, AggregateFunctionSet system_catalog.CreateFunction(data, info); } +void ExtensionUtil::RegisterFunction(DatabaseInstance &db, CreateSecretFunction function) { + D_ASSERT(!function.secret_type.empty()); + auto &config = DBConfig::GetConfig(db); + config.secret_manager->RegisterSecretFunction(std::move(function), OnCreateConflict::ERROR_ON_CONFLICT); +} + void ExtensionUtil::RegisterFunction(DatabaseInstance &db, TableFunction function) { D_ASSERT(!function.name.empty()); TableFunctionSet set(function.name); @@ -152,6 +159,11 @@ void ExtensionUtil::RegisterType(DatabaseInstance &db, string type_name, Logical system_catalog.CreateType(data, info); } +void ExtensionUtil::RegisterSecretType(DatabaseInstance &db, SecretType secret_type) { + auto &config = DBConfig::GetConfig(db); + config.secret_manager->RegisterSecretType(secret_type); +} + void ExtensionUtil::RegisterCastFunction(DatabaseInstance &db, const LogicalType &source, const LogicalType &target, BoundCastInfo function, int64_t implicit_cast_cost) { auto &config = DBConfig::GetConfig(db); diff --git a/src/duckdb/src/main/materialized_query_result.cpp b/src/duckdb/src/main/materialized_query_result.cpp index a3a5d9b3f..85bff412d 100644 --- a/src/duckdb/src/main/materialized_query_result.cpp +++ b/src/duckdb/src/main/materialized_query_result.cpp @@ -13,7 +13,7 @@ MaterializedQueryResult::MaterializedQueryResult(StatementType statement_type, S collection(std::move(collection_p)), scan_initialized(false) { } -MaterializedQueryResult::MaterializedQueryResult(PreservedError error) +MaterializedQueryResult::MaterializedQueryResult(ErrorData error) : QueryResult(QueryResultType::MATERIALIZED_RESULT, std::move(error)), scan_initialized(false) { } diff --git a/src/duckdb/src/main/pending_query_result.cpp b/src/duckdb/src/main/pending_query_result.cpp index 02501ca4c..89ec1c451 100644 --- a/src/duckdb/src/main/pending_query_result.cpp +++ b/src/duckdb/src/main/pending_query_result.cpp @@ -11,7 +11,7 @@ PendingQueryResult::PendingQueryResult(shared_ptr context_p, Prep context(std::move(context_p)), allow_stream_result(allow_stream_result) { } -PendingQueryResult::PendingQueryResult(PreservedError error) +PendingQueryResult::PendingQueryResult(ErrorData error) : BaseQueryResult(QueryResultType::PENDING_RESULT, std::move(error)) { } diff --git a/src/duckdb/src/main/prepared_statement.cpp b/src/duckdb/src/main/prepared_statement.cpp index a50997c28..df955171d 100644 --- a/src/duckdb/src/main/prepared_statement.cpp +++ b/src/duckdb/src/main/prepared_statement.cpp @@ -12,7 +12,7 @@ PreparedStatement::PreparedStatement(shared_ptr context, shared_p D_ASSERT(data || !success); } -PreparedStatement::PreparedStatement(PreservedError error) : context(nullptr), success(false), error(std::move(error)) { +PreparedStatement::PreparedStatement(ErrorData error) : context(nullptr), success(false), error(std::move(error)) { } PreparedStatement::~PreparedStatement() { @@ -23,7 +23,7 @@ const string &PreparedStatement::GetError() { return error.Message(); } -PreservedError &PreparedStatement::GetErrorObject() { +ErrorData &PreparedStatement::GetErrorObject() { return error; } @@ -98,15 +98,15 @@ unique_ptr PreparedStatement::PendingQuery(case_insensitive_ bool allow_stream_result) { if (!success) { auto exception = InvalidInputException("Attempting to execute an unsuccessfully prepared statement!"); - return make_uniq(PreservedError(exception)); + return make_uniq(ErrorData(exception)); } PendingQueryParameters parameters; parameters.parameters = &named_values; try { VerifyParameters(named_values, named_param_map); - } catch (const Exception &ex) { - return make_uniq(PreservedError(ex)); + } catch (const std::exception &ex) { + return make_uniq(ErrorData(ex)); } D_ASSERT(data); diff --git a/src/duckdb/src/main/prepared_statement_data.cpp b/src/duckdb/src/main/prepared_statement_data.cpp index 4bc356ce8..ed129d426 100644 --- a/src/duckdb/src/main/prepared_statement_data.cpp +++ b/src/duckdb/src/main/prepared_statement_data.cpp @@ -1,6 +1,7 @@ #include "duckdb/main/prepared_statement_data.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/parser/sql_statement.hpp" +#include "duckdb/common/exception/binder_exception.hpp" namespace duckdb { @@ -25,6 +26,10 @@ bool PreparedStatementData::RequireRebind(ClientContext &context, optional_ptrfind(identifier); - D_ASSERT(lookup != values->end()); + if (lookup == values->end()) { + break; + } if (lookup->second.type() != it.second->return_type) { return true; } diff --git a/src/duckdb/src/main/query_profiler.cpp b/src/duckdb/src/main/query_profiler.cpp index 26b17f046..86c1c0267 100644 --- a/src/duckdb/src/main/query_profiler.cpp +++ b/src/duckdb/src/main/query_profiler.cpp @@ -9,7 +9,7 @@ #include "duckdb/common/tree_renderer.hpp" #include "duckdb/execution/expression_executor.hpp" #include "duckdb/execution/operator/helper/physical_execute.hpp" -#include "duckdb/execution/operator/join/physical_delim_join.hpp" +#include "duckdb/execution/operator/join/physical_left_delim_join.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/main/client_config.hpp" #include "duckdb/main/client_context.hpp" @@ -101,7 +101,8 @@ bool QueryProfiler::OperatorRequiresProfiling(PhysicalOperatorType op_type) { case PhysicalOperatorType::CROSS_PRODUCT: case PhysicalOperatorType::PIECEWISE_MERGE_JOIN: case PhysicalOperatorType::IE_JOIN: - case PhysicalOperatorType::DELIM_JOIN: + case PhysicalOperatorType::LEFT_DELIM_JOIN: + case PhysicalOperatorType::RIGHT_DELIM_JOIN: case PhysicalOperatorType::UNION: case PhysicalOperatorType::RECURSIVE_CTE: case PhysicalOperatorType::EMPTY_RESULT: @@ -381,15 +382,14 @@ void QueryProfiler::QueryTreeToStream(std::ostream &ss) const { return; } - if (context.client_data->http_state && !context.client_data->http_state->IsEmpty()) { - string read = - "in: " + StringUtil::BytesToHumanReadableString(context.client_data->http_state->total_bytes_received); - string written = - "out: " + StringUtil::BytesToHumanReadableString(context.client_data->http_state->total_bytes_sent); - string head = "#HEAD: " + to_string(context.client_data->http_state->head_count); - string get = "#GET: " + to_string(context.client_data->http_state->get_count); - string put = "#PUT: " + to_string(context.client_data->http_state->put_count); - string post = "#POST: " + to_string(context.client_data->http_state->post_count); + auto http_state = HTTPState::TryGetState(context, false); + if (http_state && !http_state->IsEmpty()) { + string read = "in: " + StringUtil::BytesToHumanReadableString(http_state->total_bytes_received); + string written = "out: " + StringUtil::BytesToHumanReadableString(http_state->total_bytes_sent); + string head = "#HEAD: " + to_string(http_state->head_count); + string get = "#GET: " + to_string(http_state->get_count); + string put = "#PUT: " + to_string(http_state->put_count); + string post = "#POST: " + to_string(http_state->post_count); constexpr idx_t TOTAL_BOX_WIDTH = 39; ss << "┌─────────────────────────────────────┐\n"; diff --git a/src/duckdb/src/main/query_result.cpp b/src/duckdb/src/main/query_result.cpp index d5edc4b97..1c96468b0 100644 --- a/src/duckdb/src/main/query_result.cpp +++ b/src/duckdb/src/main/query_result.cpp @@ -13,7 +13,7 @@ BaseQueryResult::BaseQueryResult(QueryResultType type, StatementType statement_t D_ASSERT(types.size() == names.size()); } -BaseQueryResult::BaseQueryResult(QueryResultType type, PreservedError error) +BaseQueryResult::BaseQueryResult(QueryResultType type, ErrorData error) : type(type), success(false), error(std::move(error)) { } @@ -25,13 +25,13 @@ void BaseQueryResult::ThrowError(const string &prepended_message) const { error.Throw(prepended_message); } -void BaseQueryResult::SetError(PreservedError error) { - success = !error; +void BaseQueryResult::SetError(ErrorData error) { + success = !error.HasError(); this->error = std::move(error); } bool BaseQueryResult::HasError() const { - D_ASSERT((bool)error == !success); + D_ASSERT(error.HasError() == !success); return !success; } @@ -44,7 +44,7 @@ const std::string &BaseQueryResult::GetError() { return error.Message(); } -PreservedError &BaseQueryResult::GetErrorObject() { +ErrorData &BaseQueryResult::GetErrorObject() { return error; } @@ -58,7 +58,7 @@ QueryResult::QueryResult(QueryResultType type, StatementType statement_type, Sta client_properties(std::move(client_properties_p)) { } -QueryResult::QueryResult(QueryResultType type, PreservedError error) +QueryResult::QueryResult(QueryResultType type, ErrorData error) : BaseQueryResult(type, std::move(error)), client_properties("UTC", ArrowOffsetSize::REGULAR) { } diff --git a/src/duckdb/src/main/relation.cpp b/src/duckdb/src/main/relation.cpp index cd5775089..970ab4a58 100644 --- a/src/duckdb/src/main/relation.cpp +++ b/src/duckdb/src/main/relation.cpp @@ -334,11 +334,11 @@ unique_ptr Relation::Explain(ExplainType type) { } void Relation::Update(const string &update, const string &condition) { - throw Exception("UPDATE can only be used on base tables!"); + throw InvalidInputException("UPDATE can only be used on base tables!"); } void Relation::Delete(const string &condition) { - throw Exception("DELETE can only be used on base tables!"); + throw InvalidInputException("DELETE can only be used on base tables!"); } shared_ptr Relation::TableFunction(const std::string &fname, const vector &values, diff --git a/src/duckdb/src/main/relation/cross_product_relation.cpp b/src/duckdb/src/main/relation/cross_product_relation.cpp index f4bc12a3d..ca3abfbed 100644 --- a/src/duckdb/src/main/relation/cross_product_relation.cpp +++ b/src/duckdb/src/main/relation/cross_product_relation.cpp @@ -11,7 +11,7 @@ CrossProductRelation::CrossProductRelation(shared_ptr left_p, shared_p : Relation(left_p->context, RelationType::CROSS_PRODUCT_RELATION), left(std::move(left_p)), right(std::move(right_p)), ref_type(ref_type) { if (left->context.GetContext() != right->context.GetContext()) { - throw Exception("Cannot combine LEFT and RIGHT relations of different connections!"); + throw InvalidInputException("Cannot combine LEFT and RIGHT relations of different connections!"); } context.GetContext()->TryBindRelation(*this, this->columns); } diff --git a/src/duckdb/src/main/relation/join_relation.cpp b/src/duckdb/src/main/relation/join_relation.cpp index 22691cd8f..e1bb0e9d5 100644 --- a/src/duckdb/src/main/relation/join_relation.cpp +++ b/src/duckdb/src/main/relation/join_relation.cpp @@ -12,7 +12,7 @@ JoinRelation::JoinRelation(shared_ptr left_p, shared_ptr rig : Relation(left_p->context, RelationType::JOIN_RELATION), left(std::move(left_p)), right(std::move(right_p)), condition(std::move(condition_p)), join_type(type), join_ref_type(join_ref_type) { if (left->context.GetContext() != right->context.GetContext()) { - throw Exception("Cannot combine LEFT and RIGHT relations of different connections!"); + throw InvalidInputException("Cannot combine LEFT and RIGHT relations of different connections!"); } context.GetContext()->TryBindRelation(*this, this->columns); } @@ -22,7 +22,7 @@ JoinRelation::JoinRelation(shared_ptr left_p, shared_ptr rig : Relation(left_p->context, RelationType::JOIN_RELATION), left(std::move(left_p)), right(std::move(right_p)), using_columns(std::move(using_columns_p)), join_type(type), join_ref_type(join_ref_type) { if (left->context.GetContext() != right->context.GetContext()) { - throw Exception("Cannot combine LEFT and RIGHT relations of different connections!"); + throw InvalidInputException("Cannot combine LEFT and RIGHT relations of different connections!"); } context.GetContext()->TryBindRelation(*this, this->columns); } diff --git a/src/duckdb/src/main/relation/read_csv_relation.cpp b/src/duckdb/src/main/relation/read_csv_relation.cpp index 3914e0ec2..a885ae0f6 100644 --- a/src/duckdb/src/main/relation/read_csv_relation.cpp +++ b/src/duckdb/src/main/relation/read_csv_relation.cpp @@ -1,14 +1,13 @@ #include "duckdb/main/relation/read_csv_relation.hpp" -#include "duckdb/execution/operator/scan/csv/buffered_csv_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_buffer_manager.hpp" -#include "duckdb/execution/operator/scan/csv/csv_sniffer.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_buffer_manager.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_sniffer.hpp" #include "duckdb/parser/expression/columnref_expression.hpp" #include "duckdb/parser/expression/comparison_expression.hpp" #include "duckdb/parser/expression/constant_expression.hpp" #include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/common/string_util.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "duckdb/common/multi_file_reader.hpp" #include "duckdb/parser/expression/star_expression.hpp" #include "duckdb/parser/query_node/select_node.hpp" @@ -52,10 +51,9 @@ ReadCSVRelation::ReadCSVRelation(const std::shared_ptr &context, csv_options.FromNamedParameters(options, *context, unused_types, unused_names); // 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; - CSVSniffer sniffer(csv_options, buffer_manager, state_machine_cache); + + auto buffer_manager = make_shared(*context, csv_options, files[0], 0); + CSVSniffer sniffer(csv_options, buffer_manager, CSVStateMachineCache::Get(*context)); auto sniffer_result = sniffer.SniffCSV(); auto &types = sniffer_result.return_types; auto &names = sniffer_result.names; diff --git a/src/duckdb/src/main/relation/setop_relation.cpp b/src/duckdb/src/main/relation/setop_relation.cpp index 2c0bf3b2d..63b47480c 100644 --- a/src/duckdb/src/main/relation/setop_relation.cpp +++ b/src/duckdb/src/main/relation/setop_relation.cpp @@ -10,7 +10,7 @@ SetOpRelation::SetOpRelation(shared_ptr left_p, shared_ptr r : Relation(left_p->context, RelationType::SET_OPERATION_RELATION), left(std::move(left_p)), right(std::move(right_p)), setop_type(setop_type_p), setop_all(setop_all) { if (left->context.GetContext() != right->context.GetContext()) { - throw Exception("Cannot combine LEFT and RIGHT relations of different connections!"); + throw InvalidInputException("Cannot combine LEFT and RIGHT relations of different connections!"); } context.GetContext()->TryBindRelation(*this, this->columns); } diff --git a/src/duckdb/src/main/secret/secret.cpp b/src/duckdb/src/main/secret/secret.cpp new file mode 100644 index 000000000..bf92728f8 --- /dev/null +++ b/src/duckdb/src/main/secret/secret.cpp @@ -0,0 +1,135 @@ +#include "duckdb/main/secret/secret.hpp" +#include "duckdb/parser/parsed_data/create_info.hpp" +#include "duckdb/planner/logical_operator.hpp" +#include "duckdb/common/case_insensitive_map.hpp" +#include "duckdb/common/pair.hpp" + +namespace duckdb { + +int64_t BaseSecret::MatchScore(const string &path) const { + int64_t longest_match = NumericLimits::Minimum(); + for (const auto &prefix : prefix_paths) { + // Handle empty scope which matches all at lowest possible score + if (prefix.empty()) { + longest_match = 0; + continue; + } + if (StringUtil::StartsWith(path, prefix)) { + longest_match = MaxValue(prefix.length(), longest_match); + } + } + return longest_match; +} + +void BaseSecret::SerializeBaseSecret(Serializer &serializer) const { + serializer.WriteProperty(100, "type", type); + serializer.WriteProperty(101, "provider", provider); + serializer.WriteProperty(102, "name", name); + serializer.WriteList(103, "scope", prefix_paths.size(), + [&](Serializer::List &list, idx_t i) { list.WriteElement(prefix_paths[i]); }); +} + +string BaseSecret::ToString(SecretDisplayType display_type) const { + return ""; +} + +void BaseSecret::Serialize(Serializer &serializer) const { + throw InternalException("Attempted to serialize secret without serialize"); +} + +string KeyValueSecret::ToString(SecretDisplayType mode) const { + string result; + + result += "name=" + name + ";"; + result += "type=" + type + ";"; + result += "provider=" + provider + ";"; + result += string("serializable=") + (serializable ? "true" : "false") + ";"; + result += "scope="; + for (const auto &scope_it : prefix_paths) { + result += scope_it + ","; + } + result = result.substr(0, result.size() - 1); + result += ";"; + for (auto it = secret_map.begin(); it != secret_map.end(); it++) { + result.append(it->first); + result.append("="); + if (mode == SecretDisplayType::REDACTED && redact_keys.find(it->first) != redact_keys.end()) { + result.append("redacted"); + } else { + result.append(it->second.ToString()); + } + if (it != --secret_map.end()) { + result.append(";"); + } + } + + return result; +} + +// FIXME: use serialization scripts +void KeyValueSecret::Serialize(Serializer &serializer) const { + BaseSecret::SerializeBaseSecret(serializer); + + vector map_values; + for (auto it = secret_map.begin(); it != secret_map.end(); it++) { + child_list_t map_struct; + map_struct.push_back(make_pair("key", Value(it->first))); + map_struct.push_back(make_pair("value", Value(it->second))); + map_values.push_back(Value::STRUCT(map_struct)); + } + + auto map_type = LogicalType::MAP(LogicalType::VARCHAR, LogicalType::VARCHAR); + auto map = Value::MAP(ListType::GetChildType(map_type), map_values); + serializer.WriteProperty(201, "secret_map", map); + + vector redact_key_values; + for (auto it = redact_keys.begin(); it != redact_keys.end(); it++) { + redact_key_values.push_back(*it); + } + auto list = Value::LIST(LogicalType::VARCHAR, redact_key_values); + serializer.WriteProperty(202, "redact_keys", list); +} + +Value KeyValueSecret::TryGetValue(const string &key, bool error_on_missing) const { + auto lookup = secret_map.find(key); + if (lookup == secret_map.end()) { + if (error_on_missing) { + throw InternalException("Failed to fetch key '%s' from secret '%s' of type '%s'", key, name, type); + } + return Value(); + } + + return lookup->second; +} + +bool CreateSecretFunctionSet::ProviderExists(const string &provider_name) { + return functions.find(provider_name) != functions.end(); +} + +void CreateSecretFunctionSet::AddFunction(CreateSecretFunction &function, OnCreateConflict on_conflict) { + if (ProviderExists(function.provider)) { + if (on_conflict == OnCreateConflict::ERROR_ON_CONFLICT) { + throw InternalException( + "Attempted to override a Create Secret Function with OnCreateConflict::ERROR_ON_CONFLICT for: '%s'", + function.provider); + } else if (on_conflict == OnCreateConflict::REPLACE_ON_CONFLICT) { + functions[function.provider] = function; + } else if (on_conflict == OnCreateConflict::ALTER_ON_CONFLICT) { + throw NotImplementedException("ALTER_ON_CONFLICT not implemented for CreateSecretFunctionSet"); + } + } else { + functions[function.provider] = function; + } +} + +CreateSecretFunction &CreateSecretFunctionSet::GetFunction(const string &provider) { + const auto &lookup = functions.find(provider); + + if (lookup == functions.end()) { + throw InternalException("Could not find Create Secret Function with provider %s"); + } + + return lookup->second; +} + +} // namespace duckdb diff --git a/src/duckdb/src/main/secret/secret_manager.cpp b/src/duckdb/src/main/secret/secret_manager.cpp new file mode 100644 index 000000000..a1faa37e7 --- /dev/null +++ b/src/duckdb/src/main/secret/secret_manager.cpp @@ -0,0 +1,631 @@ +#include "duckdb/main/secret/secret_manager.hpp" + +#include "duckdb/catalog/catalog_entry.hpp" +#include "duckdb/common/common.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/local_file_system.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/serializer/binary_deserializer.hpp" +#include "duckdb/common/serializer/binary_serializer.hpp" +#include "duckdb/common/serializer/buffered_file_reader.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/extension_helper.hpp" +#include "duckdb/main/secret/secret_storage.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" +#include "duckdb/parser/statement/create_statement.hpp" +#include "duckdb/planner/operator/logical_create_secret.hpp" + +namespace duckdb { + +SecretCatalogEntry::SecretCatalogEntry(unique_ptr secret_p, Catalog &catalog) + : InCatalogEntry(CatalogType::SECRET_ENTRY, catalog, secret_p->secret->GetName()), secret(std::move(secret_p)) { + internal = true; +} + +SecretCatalogEntry::SecretCatalogEntry(unique_ptr secret_p, Catalog &catalog) + : InCatalogEntry(CatalogType::SECRET_ENTRY, catalog, secret_p->GetName()) { + internal = true; + secret = make_uniq(std::move(secret_p)); +} + +const BaseSecret &SecretMatch::GetSecret() { + return *secret_entry.get()->secret; +} + +constexpr const char *SecretManager::TEMPORARY_STORAGE_NAME; +constexpr const char *SecretManager::LOCAL_FILE_STORAGE_NAME; + +void SecretManager::Initialize(DatabaseInstance &db) { + lock_guard lck(manager_lock); + + // Construct default path + LocalFileSystem fs; + config.default_secret_path = fs.GetHomeDirectory(); + vector path_components = {".duckdb", "stored_secrets", ExtensionHelper::GetVersionDirectoryName()}; + for (auto &path_ele : path_components) { + config.default_secret_path = fs.JoinPath(config.default_secret_path, path_ele); + if (!fs.DirectoryExists(config.default_secret_path)) { + fs.CreateDirectory(config.default_secret_path); + } + } + config.secret_path = config.default_secret_path; + + // Set the defaults for persistent storage + config.default_persistent_storage = LOCAL_FILE_STORAGE_NAME; + + // Store the current db for enabling autoloading + this->db = &db; +} + +void SecretManager::LoadSecretStorage(unique_ptr storage) { + lock_guard lck(manager_lock); + return LoadSecretStorageInternal(std::move(storage)); +} + +void SecretManager::LoadSecretStorageInternal(unique_ptr storage) { + if (secret_storages.find(storage->GetName()) != secret_storages.end()) { + throw InternalException("Secret Storage with name '%s' already registered!", storage->GetName()); + } + + // Check for tie-break offset collisions to ensure we can always tie-break cleanly + for (const auto &storage_ptr : secret_storages) { + if (storage_ptr.second->GetTieBreakOffset() == storage->GetTieBreakOffset()) { + throw InternalException("Failed to load secret storage '%s', tie break score collides with '%s'", + storage->GetName(), storage_ptr.second->GetName()); + } + } + + secret_storages[storage->GetName()] = std::move(storage); +} + +// FIXME: use serialization scripts? +unique_ptr SecretManager::DeserializeSecret(Deserializer &deserializer) { + auto type = deserializer.ReadProperty(100, "type"); + auto provider = deserializer.ReadProperty(101, "provider"); + auto name = deserializer.ReadProperty(102, "name"); + vector scope; + deserializer.ReadList(103, "scope", + [&](Deserializer::List &list, idx_t i) { scope.push_back(list.ReadElement()); }); + + auto secret_type = LookupTypeInternal(type); + + if (!secret_type.deserializer) { + throw InternalException( + "Attempted to deserialize secret type '%s' which does not have a deserialization method", type); + } + + return secret_type.deserializer(deserializer, {scope, type, provider, name}); +} + +void SecretManager::RegisterSecretType(SecretType &type) { + lock_guard lck(manager_lock); + auto lookup = secret_types.find(type.name); + if (lookup != secret_types.end()) { + throw InternalException("Attempted to register an already registered secret type: '%s'", type.name); + } + secret_types[type.name] = type; +} + +void SecretManager::RegisterSecretFunction(CreateSecretFunction function, OnCreateConflict on_conflict) { + unique_lock lck(manager_lock); + auto lookup = secret_functions.find(function.secret_type); + if (lookup != secret_functions.end()) { + lookup->second.AddFunction(function, on_conflict); + return; + } + CreateSecretFunctionSet new_set(function.secret_type); + new_set.AddFunction(function, OnCreateConflict::ERROR_ON_CONFLICT); + secret_functions.insert({function.secret_type, new_set}); +} + +optional_ptr SecretManager::RegisterSecret(CatalogTransaction transaction, + unique_ptr secret, + OnCreateConflict on_conflict, SecretPersistType persist_type, + const string &storage) { + InitializeSecrets(transaction); + return RegisterSecretInternal(transaction, std::move(secret), on_conflict, persist_type, storage); +} + +optional_ptr SecretManager::RegisterSecretInternal(CatalogTransaction transaction, + unique_ptr secret, + OnCreateConflict on_conflict, + SecretPersistType persist_type, const string &storage) { + //! Ensure we only create secrets for known types; + LookupTypeInternal(secret->GetType()); + + //! Handle default for persist type + if (persist_type == SecretPersistType::DEFAULT) { + if (storage.empty()) { + persist_type = config.default_persist_type; + } else if (storage == TEMPORARY_STORAGE_NAME) { + persist_type = SecretPersistType::TEMPORARY; + } else { + persist_type = SecretPersistType::PERSISTENT; + } + } + + //! Resolve storage + string resolved_storage; + if (storage.empty()) { + resolved_storage = + persist_type == SecretPersistType::PERSISTENT ? config.default_persistent_storage : TEMPORARY_STORAGE_NAME; + } else { + resolved_storage = storage; + } + + //! Lookup which backend to store the secret in + auto backend = GetSecretStorage(resolved_storage); + if (!backend) { + throw InvalidInputException("Secret storage '%s' not found!", resolved_storage); + } + + // Validation on both allow_persistent_secrets and storage backend's own persist type + if (persist_type == SecretPersistType::PERSISTENT) { + if (backend->persistent) { + if (!config.allow_persistent_secrets) { + throw InvalidInputException( + "Persistent secrets are currently disabled. To enable them, restart duckdb and " + "run 'SET allow_persistent_secrets=true'"); + } + } else { // backend is temp + throw InvalidInputException("Cannot create persistent secrets in a temporary secret storage!"); + } + } else { // SecretPersistType::TEMPORARY + if (backend->persistent) { + throw InvalidInputException("Cannot create temporary secrets in a persistent secret storage!"); + } + } + return backend->StoreSecret(std::move(secret), on_conflict, &transaction); +} + +optional_ptr SecretManager::LookupFunctionInternal(const string &type, const string &provider) { + unique_lock lck(manager_lock); + auto lookup = secret_functions.find(type); + + if (lookup != secret_functions.end()) { + if (lookup->second.ProviderExists(provider)) { + return &lookup->second.GetFunction(provider); + } + } + + // Try autoloading + lck.unlock(); + AutoloadExtensionForFunction(type, provider); + lck.lock(); + + lookup = secret_functions.find(type); + + if (lookup != secret_functions.end()) { + if (lookup->second.ProviderExists(provider)) { + return &lookup->second.GetFunction(provider); + } + } + + return nullptr; +} + +optional_ptr SecretManager::CreateSecret(ClientContext &context, const CreateSecretInfo &info) { + // Note that a context is required for CreateSecret, as the CreateSecretFunction expects one + auto transaction = CatalogTransaction::GetSystemCatalogTransaction(context); + InitializeSecrets(transaction); + + // Make a copy to set the provider to default if necessary + CreateSecretInput function_input {info.type, info.provider, info.storage_type, info.name, info.scope, info.options}; + if (function_input.provider.empty()) { + auto secret_type = LookupTypeInternal(function_input.type); + function_input.provider = secret_type.default_provider; + } + + // Lookup function + auto function_lookup = LookupFunctionInternal(function_input.type, function_input.provider); + if (!function_lookup) { + throw InvalidInputException("Could not find CreateSecretFunction for type: '%s' and provider: '%s'", info.type, + info.provider); + } + + // Call the function + auto secret = function_lookup->function(context, function_input); + + if (!secret) { + throw InternalException("CreateSecretFunction for type: '%s' and provider: '%s' did not return a secret!", + info.type, info.provider); + } + + // Register the secret at the secret_manager + return RegisterSecretInternal(transaction, std::move(secret), info.on_conflict, info.persist_type, + info.storage_type); +} + +BoundStatement SecretManager::BindCreateSecret(CatalogTransaction transaction, CreateSecretInfo &info) { + InitializeSecrets(transaction); + + auto type = info.type; + auto provider = info.provider; + bool default_provider = false; + + if (provider.empty()) { + default_provider = true; + auto secret_type = LookupTypeInternal(type); + provider = secret_type.default_provider; + } + + string default_string = default_provider ? "default " : ""; + + auto function = LookupFunctionInternal(type, provider); + + if (!function) { + throw BinderException("Could not find create secret function for secret type '%s' with %sprovider '%s'", type, + default_string, provider); + } + + auto bound_info = info; + bound_info.options.clear(); + + // We cast the passed parameters + for (const auto ¶m : info.options) { + auto matched_param = function->named_parameters.find(param.first); + if (matched_param == function->named_parameters.end()) { + throw BinderException("Unknown parameter '%s' for secret type '%s' with %sprovider '%s'", param.first, type, + default_string, provider); + } + + // Cast the provided value to the expected type + string error_msg; + Value cast_value; + if (!param.second.DefaultTryCastAs(matched_param->second, cast_value, &error_msg)) { + throw BinderException("Failed to cast option '%s' to type '%s': '%s'", matched_param->first, + matched_param->second.ToString(), error_msg); + } + + bound_info.options[matched_param->first] = {cast_value}; + } + + BoundStatement result; + result.names = {"Success"}; + result.types = {LogicalType::BOOLEAN}; + result.plan = make_uniq(*function, std::move(bound_info)); + return result; +} + +SecretMatch SecretManager::LookupSecret(CatalogTransaction transaction, const string &path, const string &type) { + InitializeSecrets(transaction); + + int64_t best_match_score = NumericLimits::Minimum(); + optional_ptr best_match = nullptr; + + for (const auto &storage_ref : GetSecretStorages()) { + if (!storage_ref.get().IncludeInLookups()) { + continue; + } + auto match = storage_ref.get().LookupSecret(path, type, &transaction); + if (match.HasMatch() && match.score > best_match_score) { + best_match = match.secret_entry.get(); + best_match_score = match.score; + } + } + + if (best_match) { + return SecretMatch(*best_match, best_match_score); + } + + return SecretMatch(); +} + +optional_ptr SecretManager::GetSecretByName(CatalogTransaction transaction, const string &name, + const string &storage) { + InitializeSecrets(transaction); + + optional_ptr result; + bool found = false; + + if (!storage.empty()) { + auto storage_lookup = GetSecretStorage(storage); + + if (!storage_lookup) { + throw InvalidInputException("Unknown secret storage found: '%s'", storage); + } + + return storage_lookup->GetSecretByName(name, &transaction); + } + + for (const auto &storage_ref : GetSecretStorages()) { + auto lookup = storage_ref.get().GetSecretByName(name, &transaction); + if (lookup) { + if (found) { + throw InternalException( + "Ambiguity detected for secret name '%s', secret occurs in multiple storage backends.", name); + } + + result = lookup; + found = true; + } + } + + return result; +} + +void SecretManager::DropSecretByName(CatalogTransaction transaction, const string &name, + OnEntryNotFound on_entry_not_found, SecretPersistType persist_type, + const string &storage) { + InitializeSecrets(transaction); + + vector> matches; + + // storage to drop from was specified directly + if (!storage.empty()) { + auto storage_lookup = GetSecretStorage(storage); + if (!storage_lookup) { + throw InvalidInputException("Unknown storage type found for drop secret: '%s'", storage); + } + matches.push_back(*storage_lookup.get()); + } else { + for (const auto &storage_ref : GetSecretStorages()) { + if (persist_type == SecretPersistType::PERSISTENT && !storage_ref.get().Persistent()) { + continue; + } + if (persist_type == SecretPersistType::TEMPORARY && storage_ref.get().Persistent()) { + continue; + } + + auto lookup = storage_ref.get().GetSecretByName(name, &transaction); + if (lookup) { + matches.push_back(storage_ref.get()); + } + } + } + + if (matches.size() > 1) { + string list_of_matches; + for (const auto &match : matches) { + list_of_matches += match.get().GetName() + ","; + } + list_of_matches.pop_back(); // trailing comma + + throw InvalidInputException( + "Ambiguity found for secret name '%s', secret occurs in multiple storages: [%s] Please specify which " + "secret to drop using: 'DROP SECRET [FROM ]'.", + name, list_of_matches); + } + + if (matches.empty()) { + if (on_entry_not_found == OnEntryNotFound::THROW_EXCEPTION) { + string storage_str; + if (!storage.empty()) { + storage_str = " for storage '" + storage + "'"; + } + throw InvalidInputException("Failed to remove non-existent secret with name '%s'%s", name, storage_str); + } + // Do nothing on OnEntryNotFound::RETURN_NULL... + } else { + matches[0].get().DropSecretByName(name, on_entry_not_found, &transaction); + } +} + +SecretType SecretManager::LookupType(const string &type) { + return LookupTypeInternal(type); +} + +SecretType SecretManager::LookupTypeInternal(const string &type) { + unique_lock lck(manager_lock); + auto lookup = secret_types.find(type); + if (lookup != secret_types.end()) { + return lookup->second; + } + + // Try autoloading + lck.unlock(); + AutoloadExtensionForType(type); + lck.lock(); + + lookup = secret_types.find(type); + if (lookup != secret_types.end()) { + return lookup->second; + } + + throw InvalidInputException("Secret type '%s' not found", type); +} + +vector> SecretManager::AllSecrets(CatalogTransaction transaction) { + InitializeSecrets(transaction); + + vector> result; + + // Add results from all backends to the result set + for (const auto &backend : secret_storages) { + auto backend_result = backend.second->AllSecrets(&transaction); + for (const auto &it : backend_result) { + result.push_back(it); + } + } + + return result; +} + +void SecretManager::ThrowOnSettingChangeIfInitialized() { + if (initialized) { + throw InvalidInputException( + "Changing Secret Manager settings after the secret manager is used is not allowed!"); + } +} + +void SecretManager::SetEnablePersistentSecrets(bool enabled) { + ThrowOnSettingChangeIfInitialized(); + config.allow_persistent_secrets = enabled; +} + +void SecretManager::ResetEnablePersistentSecrets() { + ThrowOnSettingChangeIfInitialized(); + config.allow_persistent_secrets = SecretManagerConfig::DEFAULT_ALLOW_PERSISTENT_SECRETS; +} + +bool SecretManager::PersistentSecretsEnabled() { + return config.allow_persistent_secrets; +} + +void SecretManager::SetDefaultStorage(const string &storage) { + ThrowOnSettingChangeIfInitialized(); + config.default_persistent_storage = storage; +} + +void SecretManager::ResetDefaultStorage() { + ThrowOnSettingChangeIfInitialized(); + config.default_persistent_storage = SecretManager::LOCAL_FILE_STORAGE_NAME; +} + +string SecretManager::DefaultStorage() { + return config.default_persistent_storage; +} + +void SecretManager::SetPersistentSecretPath(const string &path) { + ThrowOnSettingChangeIfInitialized(); + config.secret_path = path; +} + +void SecretManager::ResetPersistentSecretPath() { + ThrowOnSettingChangeIfInitialized(); + config.secret_path = config.default_secret_path; +} + +string SecretManager::PersistentSecretPath() { + return config.secret_path; +} + +void SecretManager::InitializeSecrets(CatalogTransaction transaction) { + if (!initialized) { + lock_guard lck(manager_lock); + if (initialized) { + // some sneaky other thread beat us to it + return; + } + + // load the tmp storage + LoadSecretStorageInternal(make_uniq(TEMPORARY_STORAGE_NAME, *transaction.db)); + + // load the persistent storage if enabled + LoadSecretStorageInternal( + make_uniq(*this, *transaction.db, LOCAL_FILE_STORAGE_NAME, config.secret_path)); + + initialized = true; + } +} + +void SecretManager::AutoloadExtensionForType(const string &type) { + ExtensionHelper::TryAutoloadFromEntry(*db, type, EXTENSION_SECRET_TYPES); +} + +void SecretManager::AutoloadExtensionForFunction(const string &type, const string &provider) { + ExtensionHelper::TryAutoloadFromEntry(*db, type + "/" + provider, EXTENSION_SECRET_PROVIDERS); +} + +optional_ptr SecretManager::GetSecretStorage(const string &name) { + lock_guard lock(manager_lock); + + auto lookup = secret_storages.find(name); + if (lookup != secret_storages.end()) { + return lookup->second.get(); + } + + return nullptr; +} + +vector> SecretManager::GetSecretStorages() { + lock_guard lock(manager_lock); + + vector> result; + + for (const auto &storage : secret_storages) { + result.push_back(*storage.second); + } + + return result; +} + +DefaultSecretGenerator::DefaultSecretGenerator(Catalog &catalog, SecretManager &secret_manager, + case_insensitive_set_t &persistent_secrets) + : DefaultGenerator(catalog), secret_manager(secret_manager), persistent_secrets(persistent_secrets) { +} + +unique_ptr DefaultSecretGenerator::CreateDefaultEntry(ClientContext &context, const string &entry_name) { + + auto secret_lu = persistent_secrets.find(entry_name); + if (secret_lu == persistent_secrets.end()) { + return nullptr; + } + + LocalFileSystem fs; + auto &catalog = Catalog::GetSystemCatalog(context); + + string base_secret_path = secret_manager.PersistentSecretPath(); + string secret_path = fs.JoinPath(base_secret_path, entry_name + ".duckdb_secret"); + + // Note each file should contain 1 secret + try { + auto file_reader = BufferedFileReader(fs, secret_path.c_str()); + if (!file_reader.Finished()) { + BinaryDeserializer deserializer(file_reader); + + deserializer.Begin(); + auto deserialized_secret = secret_manager.DeserializeSecret(deserializer); + deserializer.End(); + + auto name = deserialized_secret->GetName(); + auto entry = make_uniq(std::move(deserialized_secret), catalog); + entry->secret->storage_mode = SecretManager::LOCAL_FILE_STORAGE_NAME; + entry->secret->persist_type = SecretPersistType::PERSISTENT; + + // Finally: we remove the default entry from the persistent_secrets, otherwise we aren't able to drop it + // later + persistent_secrets.erase(secret_lu); + + return std::move(entry); + } + } catch (std::exception &ex) { + ErrorData error(ex); + switch (error.Type()) { + case ExceptionType::SERIALIZATION: + throw SerializationException( + "Failed to deserialize the persistent secret file: '%s'. The file maybe be " + "corrupt, please remove the file, restart and try again. (error message: '%s')", + secret_path, error.RawMessage()); + case ExceptionType::IO: + throw IOException( + "Failed to open the persistent secret file: '%s'. Some other process may have removed it, " + "please restart and try again. (error message: '%s')", + secret_path, error.RawMessage()); + default: + throw; + } + } + + throw SerializationException("Failed to deserialize secret '%s' from '%s': file appears empty! Please remove the " + "file, restart and try again", + entry_name, secret_path); +} + +vector DefaultSecretGenerator::GetDefaultEntries() { + vector ret; + + for (const auto &res : persistent_secrets) { + ret.push_back(res); + } + + return ret; +} + +SecretManager &SecretManager::Get(ClientContext &context) { + return *DBConfig::GetConfig(context).secret_manager; +} +SecretManager &SecretManager::Get(DatabaseInstance &db) { + return *DBConfig::GetConfig(db).secret_manager; +} + +void SecretManager::DropSecretByName(ClientContext &context, const string &name, OnEntryNotFound on_entry_not_found, + SecretPersistType persist_type, const string &storage) { + auto transaction = CatalogTransaction::GetSystemCatalogTransaction(context); + return DropSecretByName(transaction, name, on_entry_not_found, persist_type, storage); +} + +} // namespace duckdb diff --git a/src/duckdb/src/main/secret/secret_storage.cpp b/src/duckdb/src/main/secret/secret_storage.cpp new file mode 100644 index 000000000..83b53c47f --- /dev/null +++ b/src/duckdb/src/main/secret/secret_storage.cpp @@ -0,0 +1,207 @@ +#include "duckdb/common/common.hpp" +#include "duckdb/common/file_system.hpp" +#include "duckdb/common/local_file_system.hpp" +#include "duckdb/common/mutex.hpp" +#include "duckdb/common/serializer/binary_serializer.hpp" +#include "duckdb/common/serializer/buffered_file_reader.hpp" +#include "duckdb/function/function_set.hpp" +#include "duckdb/main/client_context.hpp" +#include "duckdb/main/extension_helper.hpp" +#include "duckdb/main/secret/secret_storage.hpp" +#include "duckdb/main/secret/secret_manager.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" +#include "duckdb/parser/statement/create_statement.hpp" + +namespace duckdb { + +SecretMatch SecretStorage::SelectBestMatch(SecretEntry &secret_entry, const string &path, SecretMatch ¤t_best) { + // Get secret match score + auto match_score = secret_entry.secret->MatchScore(path); + + // On no match + if (match_score == NumericLimits::Minimum()) { + return current_best; + } + + // The number of characters that match, where 0 means matching the catchall of "*" + D_ASSERT(match_score >= 0); + + // Apply storage tie-break offset + match_score = OffsetMatchScore(match_score); + + // Choose the best matching score, tie-breaking on secret name when necessary + if (match_score > current_best.score) { + return SecretMatch(secret_entry, match_score); + } else if (match_score == current_best.score && + secret_entry.secret->GetName() < current_best.GetSecret().GetName()) { + return SecretMatch(secret_entry, match_score); + } else { + return current_best; + } +} + +optional_ptr CatalogSetSecretStorage::StoreSecret(unique_ptr secret, + OnCreateConflict on_conflict, + optional_ptr transaction) { + if (secrets->GetEntry(GetTransactionOrDefault(transaction), secret->GetName())) { + if (on_conflict == OnCreateConflict::ERROR_ON_CONFLICT) { + string persist_string = persistent ? "Persistent" : "Temporary"; + string storage_string = persistent ? " in secret storage '" + storage_name + "'" : ""; + throw InvalidInputException("%s secret with name '%s' already exists%s!", persist_string, secret->GetName(), + storage_string); + } else if (on_conflict == OnCreateConflict::IGNORE_ON_CONFLICT) { + return nullptr; + } else if (on_conflict == OnCreateConflict::ALTER_ON_CONFLICT) { + throw InternalException("unknown OnCreateConflict found while registering secret"); + } else if (on_conflict == OnCreateConflict::REPLACE_ON_CONFLICT) { + secrets->DropEntry(GetTransactionOrDefault(transaction), secret->GetName(), true, true); + } + } + + // Call write function + WriteSecret(*secret, on_conflict); + + auto secret_name = secret->GetName(); + auto secret_entry = make_uniq(std::move(secret), Catalog::GetSystemCatalog(db)); + secret_entry->temporary = !persistent; + secret_entry->secret->storage_mode = storage_name; + secret_entry->secret->persist_type = persistent ? SecretPersistType::PERSISTENT : SecretPersistType::TEMPORARY; + DependencyList l; + secrets->CreateEntry(GetTransactionOrDefault(transaction), secret_name, std::move(secret_entry), l); + + auto secret_catalog_entry = + &secrets->GetEntry(GetTransactionOrDefault(transaction), secret_name)->Cast(); + return secret_catalog_entry->secret; +} + +vector> CatalogSetSecretStorage::AllSecrets(optional_ptr transaction) { + vector> ret_value; + const std::function callback = [&](CatalogEntry &entry) { + auto &cast_entry = entry.Cast(); + ret_value.push_back(*cast_entry.secret); + }; + secrets->Scan(GetTransactionOrDefault(transaction), callback); + return ret_value; +} + +void CatalogSetSecretStorage::DropSecretByName(const string &name, OnEntryNotFound on_entry_not_found, + optional_ptr transaction) { + auto entry = secrets->GetEntry(GetTransactionOrDefault(transaction), name); + if (!entry && on_entry_not_found == OnEntryNotFound::THROW_EXCEPTION) { + string persist_string = persistent ? "persistent" : "temporary"; + string storage_string = persistent ? " in secret storage '" + storage_name + "'" : ""; + throw InvalidInputException("Failed to remove non-existent %s secret '%s'%s", persist_string, name, + storage_string); + } + + secrets->DropEntry(GetTransactionOrDefault(transaction), name, true, true); + RemoveSecret(name, on_entry_not_found); +} + +SecretMatch CatalogSetSecretStorage::LookupSecret(const string &path, const string &type, + optional_ptr transaction) { + auto best_match = SecretMatch(); + + const std::function callback = [&](CatalogEntry &entry) { + auto &cast_entry = entry.Cast(); + if (cast_entry.secret->secret->GetType() == type) { + best_match = SelectBestMatch(*cast_entry.secret, path, best_match); + } + }; + secrets->Scan(GetTransactionOrDefault(transaction), callback); + + if (best_match.HasMatch()) { + return best_match; + } + + return SecretMatch(); +} + +optional_ptr CatalogSetSecretStorage::GetSecretByName(const string &name, + optional_ptr transaction) { + auto res = secrets->GetEntry(GetTransactionOrDefault(transaction), name); + + if (res) { + auto &cast_entry = res->Cast(); + return cast_entry.secret; + } + + return nullptr; +} + +LocalFileSecretStorage::LocalFileSecretStorage(SecretManager &manager, DatabaseInstance &db_p, const string &name_p, + const string &secret_path) + : CatalogSetSecretStorage(db_p, name_p), secret_path(secret_path) { + persistent = true; + + LocalFileSystem fs; + + if (!fs.DirectoryExists(secret_path)) { + fs.CreateDirectory(secret_path); + } + + if (persistent_secrets.empty()) { + fs.ListFiles(secret_path, [&](const string &fname, bool is_dir) { + string full_path = fs.JoinPath(secret_path, fname); + + if (StringUtil::EndsWith(full_path, ".duckdb_secret")) { + string secret_name = fname.substr(0, fname.size() - 14); // size of file ext + persistent_secrets.insert(secret_name); + } + }); + } + + auto &catalog = Catalog::GetSystemCatalog(db); + secrets = make_uniq(Catalog::GetSystemCatalog(db), + make_uniq(catalog, manager, persistent_secrets)); +} + +void CatalogSetSecretStorage::WriteSecret(const BaseSecret &secret, OnCreateConflict on_conflict) { + // By default, this writes nothing +} +void CatalogSetSecretStorage::RemoveSecret(const string &name, OnEntryNotFound on_entry_not_found) { + // By default, this writes nothing +} + +CatalogTransaction CatalogSetSecretStorage::GetTransactionOrDefault(optional_ptr transaction) { + if (transaction) { + return *transaction; + } + return CatalogTransaction::GetSystemTransaction(db); +} + +void LocalFileSecretStorage::WriteSecret(const BaseSecret &secret, OnCreateConflict on_conflict) { + LocalFileSystem fs; + auto file_path = fs.JoinPath(secret_path, secret.GetName() + ".duckdb_secret"); + + if (fs.FileExists(file_path)) { + fs.RemoveFile(file_path); + } + + auto file_writer = BufferedFileWriter(fs, file_path); + + auto serializer = BinarySerializer(file_writer); + serializer.Begin(); + secret.Serialize(serializer); + serializer.End(); + + file_writer.Flush(); +} + +void LocalFileSecretStorage::RemoveSecret(const string &secret, OnEntryNotFound on_entry_not_found) { + LocalFileSystem fs; + string file = fs.JoinPath(secret_path, secret + ".duckdb_secret"); + persistent_secrets.erase(secret); + try { + fs.RemoveFile(file); + } catch (std::exception &ex) { + ErrorData error(ex); + if (error.Type() == ExceptionType::IO) { + throw IOException("Failed to remove secret file '%s', the file may have been removed by another duckdb " + "instance. (original error: '%s')", + file, error.RawMessage()); + } + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/main/settings/settings.cpp b/src/duckdb/src/main/settings/settings.cpp index e75c8f500..2cecda573 100644 --- a/src/duckdb/src/main/settings/settings.cpp +++ b/src/duckdb/src/main/settings/settings.cpp @@ -9,6 +9,7 @@ #include "duckdb/main/database.hpp" #include "duckdb/main/database_manager.hpp" #include "duckdb/main/query_profiler.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/parallel/task_scheduler.hpp" #include "duckdb/parser/parser.hpp" #include "duckdb/planner/expression_binder.hpp" @@ -17,6 +18,10 @@ namespace duckdb { +const string GetDefaultUserAgent() { + return StringUtil::Format("duckdb/%s(%s)", DuckDB::LibraryVersion(), DuckDB::Platform()); +} + //===--------------------------------------------------------------------===// // Access Mode //===--------------------------------------------------------------------===// @@ -56,6 +61,22 @@ Value AccessModeSetting::GetSetting(ClientContext &context) { } } +//===--------------------------------------------------------------------===// +// Allow Persistent Secrets +//===--------------------------------------------------------------------===// +void AllowPersistentSecrets::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.secret_manager->SetEnablePersistentSecrets(input.GetValue()); +} + +void AllowPersistentSecrets::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.secret_manager->ResetEnablePersistentSecrets(); +} + +Value AllowPersistentSecrets::GetSetting(ClientContext &context) { + auto &config = DBConfig::GetConfig(context); + return config.secret_manager->PersistentSecretsEnabled(); +} + //===--------------------------------------------------------------------===// // Checkpoint Threshold //===--------------------------------------------------------------------===// @@ -319,6 +340,22 @@ Value DefaultNullOrderSetting::GetSetting(ClientContext &context) { } } +//===--------------------------------------------------------------------===// +// Default Null Order +//===--------------------------------------------------------------------===// +void DefaultSecretStorage::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.secret_manager->SetDefaultStorage(input.ToString()); +} + +void DefaultSecretStorage::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.secret_manager->ResetDefaultStorage(); +} + +Value DefaultSecretStorage::GetSetting(ClientContext &context) { + auto &config = DBConfig::GetConfig(context); + return config.secret_manager->DefaultStorage(); +} + //===--------------------------------------------------------------------===// // Disabled File Systems //===--------------------------------------------------------------------===// @@ -518,31 +555,33 @@ Value EnableProfilingSetting::GetSetting(ClientContext &context) { //===--------------------------------------------------------------------===// // Custom Extension Repository //===--------------------------------------------------------------------===// -void CustomExtensionRepository::ResetLocal(ClientContext &context) { - ClientConfig::GetConfig(context).custom_extension_repo = ClientConfig().custom_extension_repo; +void CustomExtensionRepository::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.options.custom_extension_repo = DBConfig().options.custom_extension_repo; } -void CustomExtensionRepository::SetLocal(ClientContext &context, const Value &input) { - ClientConfig::GetConfig(context).custom_extension_repo = StringUtil::Lower(input.ToString()); +void CustomExtensionRepository::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.options.custom_extension_repo = StringUtil::Lower(input.ToString()); } Value CustomExtensionRepository::GetSetting(ClientContext &context) { - return Value(ClientConfig::GetConfig(context).custom_extension_repo); + auto &config = DBConfig::GetConfig(context); + return Value(config.options.custom_extension_repo); } //===--------------------------------------------------------------------===// // Autoload Extension Repository //===--------------------------------------------------------------------===// -void AutoloadExtensionRepository::ResetLocal(ClientContext &context) { - ClientConfig::GetConfig(context).autoinstall_extension_repo = ClientConfig().autoinstall_extension_repo; +void AutoloadExtensionRepository::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.options.autoinstall_extension_repo = DBConfig().options.autoinstall_extension_repo; } -void AutoloadExtensionRepository::SetLocal(ClientContext &context, const Value &input) { - ClientConfig::GetConfig(context).autoinstall_extension_repo = StringUtil::Lower(input.ToString()); +void AutoloadExtensionRepository::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.options.autoinstall_extension_repo = StringUtil::Lower(input.ToString()); } Value AutoloadExtensionRepository::GetSetting(ClientContext &context) { - return Value(ClientConfig::GetConfig(context).autoinstall_extension_repo); + auto &config = DBConfig::GetConfig(context); + return Value(config.options.autoinstall_extension_repo); } //===--------------------------------------------------------------------===// @@ -614,6 +653,21 @@ Value EnableProgressBarPrintSetting::GetSetting(ClientContext &context) { return Value::BOOLEAN(ClientConfig::GetConfig(context).print_progress_bar); } +//===--------------------------------------------------------------------===// +// Errors As JSON +//===--------------------------------------------------------------------===// +void ErrorsAsJsonSetting::ResetLocal(ClientContext &context) { + ClientConfig::GetConfig(context).errors_as_json = ClientConfig().errors_as_json; +} + +void ErrorsAsJsonSetting::SetLocal(ClientContext &context, const Value &input) { + ClientConfig::GetConfig(context).errors_as_json = BooleanValue::Get(input); +} + +Value ErrorsAsJsonSetting::GetSetting(ClientContext &context) { + return Value::BOOLEAN(ClientConfig::GetConfig(context).errors_as_json ? 1 : 0); +} + //===--------------------------------------------------------------------===// // Explain Output //===--------------------------------------------------------------------===// @@ -708,6 +762,10 @@ void ForceCompressionSetting::SetGlobal(DatabaseInstance *db, DBConfig &config, config.options.force_compression = CompressionType::COMPRESSION_AUTO; } else { auto compression_type = CompressionTypeFromString(compression); + if (CompressionTypeIsDeprecated(compression_type)) { + throw ParserException("Attempted to force a deprecated compression type (%s)", + CompressionTypeToString(compression_type)); + } if (compression_type == CompressionType::COMPRESSION_AUTO) { auto compression_types = StringUtil::Join(ListCompressionTypes(), ", "); throw ParserException("Unrecognized option for PRAGMA force_compression, expected %s", compression_types); @@ -755,6 +813,11 @@ void HomeDirectorySetting::ResetLocal(ClientContext &context) { void HomeDirectorySetting::SetLocal(ClientContext &context, const Value &input) { auto &config = ClientConfig::GetConfig(context); + + if (!input.IsNull() && FileSystem::GetFileSystem(context).IsRemoteFile(input.ToString())) { + throw InvalidInputException("Cannot set the home directory to a remote path"); + } + config.home_directory = input.IsNull() ? string() : input.ToString(); } @@ -873,6 +936,22 @@ Value MaximumMemorySetting::GetSetting(ClientContext &context) { return Value(StringUtil::BytesToHumanReadableString(config.options.maximum_memory)); } +//===--------------------------------------------------------------------===// +// Old Implicit Casting +//===--------------------------------------------------------------------===// +void OldImplicitCasting::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.options.old_implicit_casting = input.GetValue(); +} + +void OldImplicitCasting::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.options.old_implicit_casting = DBConfig().options.old_implicit_casting; +} + +Value OldImplicitCasting::GetSetting(ClientContext &context) { + auto &config = DBConfig::GetConfig(context); + return Value::BOOLEAN(config.options.old_implicit_casting); +} + //===--------------------------------------------------------------------===// // Password Setting //===--------------------------------------------------------------------===// @@ -1121,6 +1200,22 @@ Value SearchPathSetting::GetSetting(ClientContext &context) { return Value(CatalogSearchEntry::ListToString(set_paths)); } +//===--------------------------------------------------------------------===// +// Secret Directory +//===--------------------------------------------------------------------===// +void SecretDirectorySetting::SetGlobal(DatabaseInstance *db, DBConfig &config, const Value &input) { + config.secret_manager->SetPersistentSecretPath(input.ToString()); +} + +void SecretDirectorySetting::ResetGlobal(DatabaseInstance *db, DBConfig &config) { + config.secret_manager->ResetPersistentSecretPath(); +} + +Value SecretDirectorySetting::GetSetting(ClientContext &context) { + auto &config = DBConfig::GetConfig(context); + return config.secret_manager->PersistentSecretPath(); +} + //===--------------------------------------------------------------------===// // Temp Directory //===--------------------------------------------------------------------===// @@ -1212,14 +1307,14 @@ void DuckDBApiSetting::SetGlobal(DatabaseInstance *db, DBConfig &config, const V if (db) { throw InvalidInputException("Cannot change duckdb_api setting while database is running"); } - config.options.duckdb_api += " " + new_value; + 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; + config.options.duckdb_api = GetDefaultUserAgent(); } Value DuckDBApiSetting::GetSetting(ClientContext &context) { diff --git a/src/duckdb/src/main/stream_query_result.cpp b/src/duckdb/src/main/stream_query_result.cpp index 003a87863..903e485e2 100644 --- a/src/duckdb/src/main/stream_query_result.cpp +++ b/src/duckdb/src/main/stream_query_result.cpp @@ -55,7 +55,12 @@ unique_ptr StreamQueryResult::FetchRaw() { { auto lock = LockContext(); CheckExecutableInternal(*lock); - chunk = context->Fetch(*lock, *this); + auto system_chunk = context->Fetch(*lock, *this); + if (system_chunk) { + chunk = make_uniq(); + chunk->Initialize(Allocator::DefaultAllocator(), system_chunk->GetTypes()); + system_chunk->Copy(*chunk, 0); + } } if (!chunk || chunk->ColumnCount() == 0 || chunk->size() == 0) { Close(); diff --git a/src/duckdb/src/optimizer/column_lifetime_analyzer.cpp b/src/duckdb/src/optimizer/column_lifetime_analyzer.cpp index 23f99f755..4d30d5f7f 100644 --- a/src/duckdb/src/optimizer/column_lifetime_analyzer.cpp +++ b/src/duckdb/src/optimizer/column_lifetime_analyzer.cpp @@ -110,6 +110,12 @@ void ColumnLifetimeAnalyzer::VisitOperator(LogicalOperator &op) { analyzer.VisitOperator(*op.children[0]); return; } + case LogicalOperatorType::LOGICAL_ORDER_BY: + // order by, for now reference all columns + // FIXME: for ORDER BY we remove columns below an ORDER BY, we just need to make sure that the projections are + // updated + everything_referenced = true; + break; case LogicalOperatorType::LOGICAL_DISTINCT: { // distinct, all projected columns are used for the DISTINCT computation // mark all columns as used and continue to the children diff --git a/src/duckdb/src/optimizer/cse_optimizer.cpp b/src/duckdb/src/optimizer/cse_optimizer.cpp index 3a17c9421..1442a9a3f 100644 --- a/src/duckdb/src/optimizer/cse_optimizer.cpp +++ b/src/duckdb/src/optimizer/cse_optimizer.cpp @@ -58,7 +58,7 @@ void CommonSubExpressionOptimizer::CountExpressions(Expression &expr, CSEReplace default: break; } - if (expr.expression_class != ExpressionClass::BOUND_AGGREGATE && !expr.HasSideEffects()) { + if (expr.expression_class != ExpressionClass::BOUND_AGGREGATE && !expr.IsVolatile()) { // we can't move aggregates to a projection, so we only consider the children of the aggregate auto node = state.expression_count.find(expr); if (node == state.expression_count.end()) { diff --git a/src/duckdb/src/optimizer/deliminator.cpp b/src/duckdb/src/optimizer/deliminator.cpp index 0e45c4d86..2771222e7 100644 --- a/src/duckdb/src/optimizer/deliminator.cpp +++ b/src/duckdb/src/optimizer/deliminator.cpp @@ -1,7 +1,5 @@ #include "duckdb/optimizer/deliminator.hpp" -#include "duckdb/optimizer/join_order/join_order_optimizer.hpp" -#include "duckdb/optimizer/remove_duplicate_groups.hpp" #include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression/bound_columnref_expression.hpp" #include "duckdb/planner/expression/bound_conjunction_expression.hpp" @@ -10,9 +8,20 @@ #include "duckdb/planner/operator/logical_comparison_join.hpp" #include "duckdb/planner/operator/logical_delim_get.hpp" #include "duckdb/planner/operator/logical_filter.hpp" +#include "duckdb/planner/operator/logical_get.hpp" +#include "duckdb/planner/table_filter.hpp" + +#include namespace duckdb { +struct JoinWithDelimGet { + JoinWithDelimGet(unique_ptr &join_p, idx_t depth_p) : join(join_p), depth(depth_p) { + } + reference> join; + idx_t depth; +}; + struct DelimCandidate { public: explicit DelimCandidate(unique_ptr &op, LogicalComparisonJoin &delim_join) @@ -22,7 +31,7 @@ struct DelimCandidate { public: unique_ptr &op; LogicalComparisonJoin &delim_join; - vector>> joins; + vector joins; idx_t delim_get_count; }; @@ -42,15 +51,30 @@ unique_ptr Deliminator::Optimize(unique_ptr op vector candidates; FindCandidates(op, candidates); + if (candidates.empty()) { + return op; + } + for (auto &candidate : candidates) { auto &delim_join = candidate.delim_join; + // Sort these so the deepest are first + std::sort(candidate.joins.begin(), candidate.joins.end(), + [](const JoinWithDelimGet &lhs, const JoinWithDelimGet &rhs) { return lhs.depth > rhs.depth; }); + bool all_removed = true; + if (!candidate.joins.empty() && HasSelection(delim_join)) { + // Keep the deepest join with DelimGet in these cases, + // as the selection can greatly reduce the cost of the RHS child of the DelimJoin + candidate.joins.erase(candidate.joins.begin()); + all_removed = false; + } + bool all_equality_conditions = true; for (auto &join : candidate.joins) { - all_removed = - RemoveJoinWithDelimGet(delim_join, candidate.delim_get_count, join, all_equality_conditions) && - all_removed; + all_removed = RemoveJoinWithDelimGet(delim_join, candidate.delim_get_count, join.join.get(), + all_equality_conditions) && + all_removed; } // Change type if there are no more duplicate-eliminated columns @@ -65,13 +89,18 @@ unique_ptr Deliminator::Optimize(unique_ptr op } } } + + // Only DelimJoins are ever created as SINGLE joins, + // and we can switch from SINGLE to LEFT if the RHS is de-duplicated by an aggr + if (delim_join.join_type == JoinType::SINGLE) { + TrySwitchSingleToLeft(delim_join); + } } return op; } void Deliminator::FindCandidates(unique_ptr &op, vector &candidates) { - // Search children before adding, so the deepest candidates get added first for (auto &child : op->children) { FindCandidates(child, candidates); } @@ -87,6 +116,33 @@ void Deliminator::FindCandidates(unique_ptr &op, vectorchildren[1], candidate); } +bool Deliminator::HasSelection(const LogicalOperator &op) { + // TODO once we implement selectivity estimation using samples we need to use that here + switch (op.type) { + case LogicalOperatorType::LOGICAL_GET: { + auto &get = op.Cast(); + for (const auto &filter : get.table_filters.filters) { + if (filter.second->filter_type != TableFilterType::IS_NOT_NULL) { + return true; + } + } + break; + } + case LogicalOperatorType::LOGICAL_FILTER: + return true; + default: + break; + } + + for (auto &child : op.children) { + if (HasSelection(*child)) { + return true; + } + } + + return false; +} + static bool OperatorIsDelimGet(LogicalOperator &op) { if (op.type == LogicalOperatorType::LOGICAL_DELIM_GET) { return true; @@ -98,20 +154,20 @@ static bool OperatorIsDelimGet(LogicalOperator &op) { return false; } -void Deliminator::FindJoinWithDelimGet(unique_ptr &op, DelimCandidate &candidate) { +void Deliminator::FindJoinWithDelimGet(unique_ptr &op, DelimCandidate &candidate, idx_t depth) { if (op->type == LogicalOperatorType::LOGICAL_DELIM_JOIN) { - FindJoinWithDelimGet(op->children[0], candidate); + FindJoinWithDelimGet(op->children[0], candidate, depth + 1); } else if (op->type == LogicalOperatorType::LOGICAL_DELIM_GET) { candidate.delim_get_count++; } else { for (auto &child : op->children) { - FindJoinWithDelimGet(child, candidate); + FindJoinWithDelimGet(child, candidate, depth + 1); } } if (op->type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN && (OperatorIsDelimGet(*op->children[0]) || OperatorIsDelimGet(*op->children[1]))) { - candidate.joins.emplace_back(op); + candidate.joins.emplace_back(op, depth); } } @@ -230,8 +286,8 @@ bool Deliminator::RemoveInequalityJoinWithDelimGet(LogicalComparisonJoin &delim_ } // TODO: we cannot perform the optimization here because our pure inequality joins don't implement - // JoinType::SINGLE yet - if (delim_join.join_type == JoinType::SINGLE) { + // JoinType::SINGLE yet, and JoinType::MARK is a special case + if (delim_join.join_type == JoinType::SINGLE || delim_join.join_type == JoinType::MARK) { bool has_one_equality = false; for (auto &cond : join_conditions) { has_one_equality = has_one_equality || IsEqualityJoinCondition(cond); @@ -283,7 +339,22 @@ bool Deliminator::RemoveInequalityJoinWithDelimGet(LogicalComparisonJoin &delim_ auto &delim_side = delim_idx == 0 ? *join_condition.left : *join_condition.right; auto &colref = delim_side.Cast(); if (colref.binding == traced_binding) { - delim_condition.comparison = FlipComparisonExpression(join_condition.comparison); + auto join_comparison = join_condition.comparison; + if (delim_condition.comparison == ExpressionType::COMPARE_DISTINCT_FROM || + delim_condition.comparison == ExpressionType::COMPARE_NOT_DISTINCT_FROM) { + // We need to compare NULL values + if (join_comparison == ExpressionType::COMPARE_EQUAL) { + join_comparison = ExpressionType::COMPARE_NOT_DISTINCT_FROM; + } else if (join_comparison == ExpressionType::COMPARE_NOTEQUAL) { + join_comparison = ExpressionType::COMPARE_DISTINCT_FROM; + } else if (join_comparison != ExpressionType::COMPARE_DISTINCT_FROM && + join_comparison != ExpressionType::COMPARE_NOT_DISTINCT_FROM) { + // The optimization does not work here + found = false; + break; + } + } + delim_condition.comparison = FlipComparisonExpression(join_comparison); found = true; break; } @@ -294,4 +365,55 @@ bool Deliminator::RemoveInequalityJoinWithDelimGet(LogicalComparisonJoin &delim_ return found_all; } +void Deliminator::TrySwitchSingleToLeft(LogicalComparisonJoin &delim_join) { + D_ASSERT(delim_join.join_type == JoinType::SINGLE); + + // Collect RHS bindings + vector join_bindings; + for (const auto &cond : delim_join.conditions) { + if (!IsEqualityJoinCondition(cond)) { + return; + } + if (cond.right->type != ExpressionType::BOUND_COLUMN_REF) { + return; + } + auto &colref = cond.right->Cast(); + join_bindings.emplace_back(colref.binding); + } + + // Now try to find an aggr in the RHS such that the join_column_bindings is a superset of the groups + reference current_op = *delim_join.children[1]; + while (current_op.get().type != LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY) { + if (current_op.get().children.size() != 1) { + return; + } + + switch (current_op.get().type) { + case LogicalOperatorType::LOGICAL_PROJECTION: + FindAndReplaceBindings(join_bindings, current_op.get().expressions, current_op.get().GetColumnBindings()); + break; + case LogicalOperatorType::LOGICAL_FILTER: + break; // Doesn't change bindings + default: + return; + } + current_op = *current_op.get().children[0]; + } + + D_ASSERT(current_op.get().type == LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY); + const auto &aggr = current_op.get().Cast(); + if (!aggr.grouping_functions.empty()) { + return; + } + + for (idx_t group_idx = 0; group_idx < aggr.groups.size(); group_idx++) { + if (std::find(join_bindings.begin(), join_bindings.end(), ColumnBinding(aggr.group_index, group_idx)) == + join_bindings.end()) { + return; + } + } + + delim_join.join_type = JoinType::LEFT; +} + } // namespace duckdb diff --git a/src/duckdb/src/optimizer/filter_combiner.cpp b/src/duckdb/src/optimizer/filter_combiner.cpp index 89df31775..ed0443796 100644 --- a/src/duckdb/src/optimizer/filter_combiner.cpp +++ b/src/duckdb/src/optimizer/filter_combiner.cpp @@ -13,6 +13,7 @@ #include "duckdb/planner/table_filter.hpp" #include "duckdb/planner/filter/constant_filter.hpp" #include "duckdb/planner/filter/null_filter.hpp" +#include "duckdb/planner/filter/struct_filter.hpp" #include "duckdb/optimizer/optimizer.hpp" namespace duckdb { @@ -391,6 +392,45 @@ bool FilterCombiner::HasFilters() { // return zonemap_checks; // } +// Try to extract a column index from a bound column ref expression, or a column ref recursively nested +// inside of a struct_extract call. If the expression is not a column ref (or nested column ref), return false. +static bool TryGetBoundColumnIndex(const vector &column_ids, const Expression &expr, idx_t &result) { + switch (expr.type) { + case ExpressionType::BOUND_COLUMN_REF: { + auto &ref = expr.Cast(); + result = column_ids[ref.binding.column_index]; + return true; + } + case ExpressionType::BOUND_FUNCTION: { + auto &func = expr.Cast(); + if (func.function.name == "struct_extract") { + auto &child_expr = func.children[0]; + return TryGetBoundColumnIndex(column_ids, *child_expr, result); + } + return false; + } + default: + return false; + } +} + +// Try to push down a filter into a expression by recursively wrapping any nested expressions in StructFilters. +// If the expression is not a struct_extract, return the inner_filter unchanged. +static unique_ptr PushDownFilterIntoExpr(const Expression &expr, unique_ptr inner_filter) { + if (expr.type == ExpressionType::BOUND_FUNCTION) { + auto &func = expr.Cast(); + if (func.function.name == "struct_extract") { + auto &child_expr = func.children[0]; + auto child_name = func.children[1]->Cast().value.GetValue(); + auto child_index = StructType::GetChildIndexUnsafe(func.children[0]->return_type, child_name); + + inner_filter = make_uniq(child_index, child_name, std::move(inner_filter)); + return PushDownFilterIntoExpr(*child_expr, std::move(inner_filter)); + } + } + return inner_filter; +} + TableFilterSet FilterCombiner::GenerateTableScanFilters(vector &column_ids) { TableFilterSet table_filters; //! First, we figure the filters that have constant expressions that we can push down to the table scan @@ -407,28 +447,35 @@ TableFilterSet FilterCombiner::GenerateTableScanFilters(vector &column_id constant_value.second[0].constant.type().InternalType() == PhysicalType::BOOL)) { //! Here we check if these filters are column references filter_exp = equivalence_map.find(constant_value.first); - if (filter_exp->second.size() == 1 && - filter_exp->second[0].get().type == ExpressionType::BOUND_COLUMN_REF) { - auto &filter_col_exp = filter_exp->second[0].get().Cast(); - auto column_index = column_ids[filter_col_exp.binding.column_index]; - if (column_index == COLUMN_IDENTIFIER_ROW_ID) { - break; - } - auto equivalence_set = filter_exp->first; - auto &entries = filter_exp->second; - auto &constant_list = constant_values.find(equivalence_set)->second; - // for each entry generate an equality expression comparing to each other - for (idx_t i = 0; i < entries.size(); i++) { - // for each entry also create a comparison with each constant - for (idx_t k = 0; k < constant_list.size(); k++) { - auto constant_filter = make_uniq(constant_value.second[k].comparison_type, - constant_value.second[k].constant); - table_filters.PushFilter(column_index, std::move(constant_filter)); - } - table_filters.PushFilter(column_index, make_uniq()); - } - equivalence_map.erase(filter_exp); + + if (filter_exp->second.size() != 1) { + continue; + } + + auto &expr = filter_exp->second[0]; + auto equiv_set = filter_exp->first; + + // Try to get the column index, either from bound column ref, or a column ref nested inside of a + // struct_extract call + idx_t column_index; + if (!TryGetBoundColumnIndex(column_ids, expr, column_index)) { + continue; + } + if (column_index == COLUMN_IDENTIFIER_ROW_ID) { + break; } + + auto &constant_list = constant_values.find(equiv_set)->second; + for (auto &constant_cmp : constant_list) { + auto constant_filter = + make_uniq(constant_cmp.comparison_type, constant_cmp.constant); + table_filters.PushFilter(column_index, PushDownFilterIntoExpr(expr, std::move(constant_filter))); + } + // We need to apply a IS NOT NULL filter to the column expression because any comparison with NULL + // is always false. + table_filters.PushFilter(column_index, PushDownFilterIntoExpr(expr, make_uniq())); + + equivalence_map.erase(filter_exp); } } } diff --git a/src/duckdb/src/optimizer/filter_pushdown.cpp b/src/duckdb/src/optimizer/filter_pushdown.cpp index 1a38ad2ae..35d756a94 100644 --- a/src/duckdb/src/optimizer/filter_pushdown.cpp +++ b/src/duckdb/src/optimizer/filter_pushdown.cpp @@ -75,6 +75,9 @@ unique_ptr FilterPushdown::PushdownJoin(unique_ptr JoinOrderOptimizer::Optimize(unique_ptrEstimateCardinality(context); auto bindings = new_logical_plan->GetColumnBindings(); auto new_stats = RelationStatisticsHelper::CombineStatsOfReorderableOperator(bindings, relation_stats); - new_stats.cardinality = MaxValue(cardinality, new_stats.cardinality); + new_stats.cardinality = cardinality; RelationStatisticsHelper::CopyRelationStats(*stats, new_stats); } diff --git a/src/duckdb/src/optimizer/join_order/query_graph_manager.cpp b/src/duckdb/src/optimizer/join_order/query_graph_manager.cpp index dddc388f2..b8835265b 100644 --- a/src/duckdb/src/optimizer/join_order/query_graph_manager.cpp +++ b/src/duckdb/src/optimizer/join_order/query_graph_manager.cpp @@ -1,14 +1,15 @@ #include "duckdb/optimizer/join_order/query_graph_manager.hpp" -#include "duckdb/planner/logical_operator.hpp" -#include "duckdb/optimizer/join_order/join_relation.hpp" + +#include "duckdb/common/assert.hpp" #include "duckdb/common/enums/join_type.hpp" -#include "duckdb/planner/operator/list.hpp" -#include "duckdb/planner/expression_iterator.hpp" -#include "duckdb/planner/expression/bound_comparison_expression.hpp" -#include "duckdb/execution/physical_plan_generator.hpp" #include "duckdb/common/printer.hpp" #include "duckdb/common/string_util.hpp" -#include "duckdb/common/assert.hpp" +#include "duckdb/execution/physical_plan_generator.hpp" +#include "duckdb/optimizer/join_order/join_relation.hpp" +#include "duckdb/planner/expression/bound_comparison_expression.hpp" +#include "duckdb/planner/expression_iterator.hpp" +#include "duckdb/planner/logical_operator.hpp" +#include "duckdb/planner/operator/list.hpp" namespace duckdb { @@ -119,7 +120,7 @@ static unique_ptr ExtractJoinRelation(unique_ptr QueryGraphManager::Reconstruct(unique_ptr plan, JoinNode &node) { @@ -334,20 +335,11 @@ unique_ptr QueryGraphManager::RewritePlan(unique_ptrhas_estimated_cardinality ? left_child->estimated_cardinality - : left_child->EstimateCardinality(context); - auto rhs_cardinality = right_child->has_estimated_cardinality ? right_child->estimated_cardinality - : right_child->EstimateCardinality(context); - if (rhs_cardinality < lhs_cardinality * cardinality_ratio) { - return; - } - std::swap(left_child, right_child); - if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { +static void FlipChildren(LogicalOperator &op) { + std::swap(op.children[0], op.children[1]); + if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN || op.type == LogicalOperatorType::LOGICAL_DELIM_JOIN) { auto &join = op.Cast(); - join.join_type = inverse; + join.join_type = InverseJoinType(join.join_type); for (auto &cond : join.conditions) { std::swap(cond.left, cond.right); cond.comparison = FlipComparisonExpression(cond.comparison); @@ -355,8 +347,21 @@ void QueryGraphManager::TryFlipChildren(LogicalOperator &op, JoinType inverse, i } if (op.type == LogicalOperatorType::LOGICAL_ANY_JOIN) { auto &join = op.Cast(); - join.join_type = inverse; + join.join_type = InverseJoinType(join.join_type); + } +} + +void QueryGraphManager::TryFlipChildren(LogicalOperator &op, idx_t cardinality_ratio) { + auto &left_child = op.children[0]; + auto &right_child = op.children[1]; + auto lhs_cardinality = left_child->has_estimated_cardinality ? left_child->estimated_cardinality + : left_child->EstimateCardinality(context); + auto rhs_cardinality = right_child->has_estimated_cardinality ? right_child->estimated_cardinality + : right_child->EstimateCardinality(context); + if (rhs_cardinality < lhs_cardinality * cardinality_ratio) { + return; } + FlipChildren(op); } unique_ptr QueryGraphManager::LeftRightOptimizations(unique_ptr input_op) { @@ -368,40 +373,52 @@ unique_ptr QueryGraphManager::LeftRightOptimizations(unique_ptr case LogicalOperatorType::LOGICAL_COMPARISON_JOIN: { auto &join = op->Cast(); - if (join.join_type == JoinType::INNER) { - TryFlipChildren(join, JoinType::INNER); - } else if (join.join_type == JoinType::LEFT && join.right_projection_map.empty()) { - TryFlipChildren(join, JoinType::RIGHT, 2); - } else if (join.join_type == JoinType::SEMI) { - idx_t has_range = 0; - if (!PhysicalPlanGenerator::HasEquality(join.conditions, has_range)) { - // if the conditions have no equality, do not flip the children. - // There is no physical join operator (yet) that can do a right_semi/anti join. - break; + switch (join.join_type) { + case JoinType::INNER: + case JoinType::OUTER: + TryFlipChildren(join); + break; + case JoinType::LEFT: + case JoinType::RIGHT: + if (join.right_projection_map.empty()) { + TryFlipChildren(join, 2); } - TryFlipChildren(join, JoinType::RIGHT_SEMI, 2); - } else if (join.join_type == JoinType::ANTI) { + break; + case JoinType::SEMI: + case JoinType::ANTI: { idx_t has_range = 0; if (!PhysicalPlanGenerator::HasEquality(join.conditions, has_range)) { // if the conditions have no equality, do not flip the children. // There is no physical join operator (yet) that can do a right_semi/anti join. break; } - TryFlipChildren(join, JoinType::RIGHT_ANTI, 2); + TryFlipChildren(join, 2); + break; + } + default: + break; } break; } case LogicalOperatorType::LOGICAL_CROSS_PRODUCT: { // cross product not a comparison join so JoinType::INNER will get ignored - TryFlipChildren(*op, JoinType::INNER, 1); + TryFlipChildren(*op, 1); break; } case LogicalOperatorType::LOGICAL_ANY_JOIN: { auto &join = op->Cast(); if (join.join_type == JoinType::LEFT && join.right_projection_map.empty()) { - TryFlipChildren(join, JoinType::RIGHT, 2); + TryFlipChildren(join, 2); } else if (join.join_type == JoinType::INNER) { - TryFlipChildren(join, JoinType::INNER, 1); + TryFlipChildren(join, 1); + } + break; + } + case LogicalOperatorType::LOGICAL_DELIM_JOIN: { + auto &join = op->Cast(); + if (HasInverseJoinType(join.join_type) && join.right_projection_map.empty()) { + FlipChildren(join); + join.delim_flipped = true; } break; } diff --git a/src/duckdb/src/optimizer/join_order/relation_manager.cpp b/src/duckdb/src/optimizer/join_order/relation_manager.cpp index d15210462..2476d2b8e 100644 --- a/src/duckdb/src/optimizer/join_order/relation_manager.cpp +++ b/src/duckdb/src/optimizer/join_order/relation_manager.cpp @@ -131,9 +131,6 @@ bool RelationManager::ExtractJoinRelations(LogicalOperator &input_op, } filter_operators.push_back(*op); } - if (op->type == LogicalOperatorType::LOGICAL_SHOW) { - return false; - } op = op->children[0].get(); } bool non_reorderable_operation = false; diff --git a/src/duckdb/src/optimizer/join_order/relation_statistics_helper.cpp b/src/duckdb/src/optimizer/join_order/relation_statistics_helper.cpp index a708f108c..94f5ddeb8 100644 --- a/src/duckdb/src/optimizer/join_order/relation_statistics_helper.cpp +++ b/src/duckdb/src/optimizer/join_order/relation_statistics_helper.cpp @@ -226,7 +226,8 @@ RelationStats RelationStatisticsHelper::CombineStatsOfNonReorderableOperator(Log idx_t child_1_card = child_stats[0].stats_initialized ? child_stats[0].cardinality : 0; idx_t child_2_card = child_stats[1].stats_initialized ? child_stats[1].cardinality : 0; ret.cardinality = MaxValue(child_1_card, child_2_card); - if (op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + switch (op.type) { + case LogicalOperatorType::LOGICAL_COMPARISON_JOIN: { auto &join = op.Cast(); switch (join.join_type) { case JoinType::RIGHT_ANTI: @@ -241,7 +242,30 @@ RelationStats RelationStatisticsHelper::CombineStatsOfNonReorderableOperator(Log default: break; } + break; + } + case LogicalOperatorType::LOGICAL_UNION: { + auto &setop = op.Cast(); + if (setop.setop_all) { + // setop returns all records + ret.cardinality = child_1_card + child_2_card; + } else { + ret.cardinality = MaxValue(child_1_card, child_2_card); + } + break; + } + case LogicalOperatorType::LOGICAL_INTERSECT: { + ret.cardinality = MinValue(child_1_card, child_2_card); + break; } + case LogicalOperatorType::LOGICAL_EXCEPT: { + ret.cardinality = child_1_card; + break; + } + default: + break; + } + ret.stats_initialized = true; ret.filter_strength = 1; ret.table_name = child_stats[0].table_name + " joined with " + child_stats[1].table_name; @@ -356,12 +380,10 @@ idx_t RelationStatisticsHelper::InspectConjunctionAND(idx_t cardinality, idx_t c continue; } auto column_count = base_stats.GetDistinctCount(); - auto filtered_card = cardinality; // column_count = 0 when there is no column count (i.e parquet scans) if (column_count > 0) { // we want the ceil of cardinality/column_count. We also want to avoid compiler errors - filtered_card = (cardinality + column_count - 1) / column_count; - cardinality_after_filters = filtered_card; + cardinality_after_filters = (cardinality + column_count - 1) / column_count; } } return cardinality_after_filters; diff --git a/src/duckdb/src/optimizer/pushdown/pushdown_projection.cpp b/src/duckdb/src/optimizer/pushdown/pushdown_projection.cpp index a0d7d8149..08432c066 100644 --- a/src/duckdb/src/optimizer/pushdown/pushdown_projection.cpp +++ b/src/duckdb/src/optimizer/pushdown/pushdown_projection.cpp @@ -6,21 +6,21 @@ namespace duckdb { -static bool HasSideEffects(LogicalProjection &proj, const unique_ptr &expr) { +static bool IsVolatile(LogicalProjection &proj, const unique_ptr &expr) { if (expr->type == ExpressionType::BOUND_COLUMN_REF) { auto &colref = expr->Cast(); D_ASSERT(colref.binding.table_index == proj.table_index); D_ASSERT(colref.binding.column_index < proj.expressions.size()); D_ASSERT(colref.depth == 0); - if (proj.expressions[colref.binding.column_index]->HasSideEffects()) { + if (proj.expressions[colref.binding.column_index]->IsVolatile()) { return true; } return false; } - bool has_side_effects = false; + bool is_volatile = false; ExpressionIterator::EnumerateChildren( - *expr, [&](unique_ptr &child) { has_side_effects |= HasSideEffects(proj, child); }); - return has_side_effects; + *expr, [&](unique_ptr &child) { is_volatile |= IsVolatile(proj, child); }); + return is_volatile; } static unique_ptr ReplaceProjectionBindings(LogicalProjection &proj, unique_ptr expr) { @@ -50,8 +50,8 @@ unique_ptr FilterPushdown::PushdownProjection(unique_ptr FilterPushdown::PushdownSemiAntiJoin(unique_ptr op) { + auto &join = op->Cast(); + if (op->type == LogicalOperatorType::LOGICAL_DELIM_JOIN) { + return FinishPushdown(std::move(op)); + } + + // push all current filters down the left side + op->children[0] = Rewrite(std::move(op->children[0])); + FilterPushdown right_pushdown(optimizer); + op->children[1] = right_pushdown.Rewrite(std::move(op->children[1])); + + bool left_empty = op->children[0]->type == LogicalOperatorType::LOGICAL_EMPTY_RESULT; + bool right_empty = op->children[1]->type == LogicalOperatorType::LOGICAL_EMPTY_RESULT; + if (left_empty && right_empty) { + // both empty: return empty result + return make_uniq(std::move(op)); + } + // TODO: if semi/anti join is created from a intersect/except statement, then we can + // push filters down into both children. + // filter pushdown happens before join order optimization, so right_anti and right_semi are not possible yet here + if (left_empty) { + // left child is empty result + switch (join.join_type) { + case JoinType::ANTI: + case JoinType::SEMI: + return make_uniq(std::move(op)); + default: + break; + } + } else if (right_empty) { + // right child is empty result + switch (join.join_type) { + case JoinType::ANTI: + // just return the left child. + return std::move(op->children[0]); + case JoinType::SEMI: + return make_uniq(std::move(op)); + default: + break; + } + } + return op; +} + +} // namespace duckdb diff --git a/src/duckdb/src/optimizer/remove_unused_columns.cpp b/src/duckdb/src/optimizer/remove_unused_columns.cpp index 4291e174f..bd71de72a 100644 --- a/src/duckdb/src/optimizer/remove_unused_columns.cpp +++ b/src/duckdb/src/optimizer/remove_unused_columns.cpp @@ -319,6 +319,27 @@ void RemoveUnusedColumns::VisitOperator(LogicalOperator &op) { } LogicalOperatorVisitor::VisitOperatorExpressions(op); LogicalOperatorVisitor::VisitOperatorChildren(op); + + if (op.type == LogicalOperatorType::LOGICAL_ASOF_JOIN || op.type == LogicalOperatorType::LOGICAL_DELIM_JOIN || + op.type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + auto &comp_join = op.Cast(); + // after removing duplicate columns we may have duplicate join conditions (if the join graph is cyclical) + vector unique_conditions; + for (auto &cond : comp_join.conditions) { + bool found = false; + for (auto &unique_cond : unique_conditions) { + if (cond.comparison == unique_cond.comparison && cond.left->Equals(*unique_cond.left) && + cond.right->Equals(*unique_cond.right)) { + found = true; + break; + } + } + if (!found) { + unique_conditions.push_back(std::move(cond)); + } + } + comp_join.conditions = std::move(unique_conditions); + } } unique_ptr RemoveUnusedColumns::VisitReplace(BoundColumnRefExpression &expr, diff --git a/src/duckdb/src/optimizer/rule/date_part_simplification.cpp b/src/duckdb/src/optimizer/rule/date_part_simplification.cpp index 037e7a635..6737e576c 100644 --- a/src/duckdb/src/optimizer/rule/date_part_simplification.cpp +++ b/src/duckdb/src/optimizer/rule/date_part_simplification.cpp @@ -92,11 +92,11 @@ unique_ptr DatePartSimplificationRule::Apply(LogicalOperator &op, ve vector> children; children.push_back(std::move(date_part.children[1])); - string error; + ErrorData error; FunctionBinder binder(rewriter.context); auto function = binder.BindScalarFunction(DEFAULT_SCHEMA, new_function_name, std::move(children), error, false); if (!function) { - throw BinderException(error); + error.Throw(); } return function; } diff --git a/src/duckdb/src/optimizer/rule/move_constants.cpp b/src/duckdb/src/optimizer/rule/move_constants.cpp index d6b4436bd..636265ff9 100644 --- a/src/duckdb/src/optimizer/rule/move_constants.cpp +++ b/src/duckdb/src/optimizer/rule/move_constants.cpp @@ -53,7 +53,7 @@ unique_ptr MoveConstantsRule::Apply(LogicalOperator &op, vector MoveConstantsRule::Apply(LogicalOperator &op, vector MoveConstantsRule::Apply(LogicalOperator &op, vector OrderedAggregateOptimizer::Apply(LogicalOperator &op, vec changes_made = true; return nullptr; } + + // Remove unnecessary ORDER BY clauses and return if nothing remains + if (aggr.order_bys->Simplify(op.Cast().groups)) { + aggr.order_bys.reset(); + changes_made = true; + return nullptr; + } + + // Rewrite first/last/arbitrary/any_value to use arg_xxx[_null] and create_sort_key + const auto &aggr_name = aggr.function.name; + string arg_xxx_name; + if (aggr_name == "last") { + arg_xxx_name = "arg_max_null"; + } else if (aggr_name == "first" || aggr_name == "arbitrary") { + arg_xxx_name = "arg_min_null"; + } else if (aggr_name == "any_value") { + arg_xxx_name = "arg_min"; + } else { + return nullptr; + } + + auto &context = rewriter.context; + FunctionBinder binder(context); + vector> sort_children; + for (auto &order : aggr.order_bys->orders) { + sort_children.emplace_back(std::move(order.expression)); + + string modifier; + modifier += (order.type == OrderType::ASCENDING) ? "ASC" : "DESC"; + modifier += " NULLS"; + modifier += (order.null_order == OrderByNullType::NULLS_FIRST) ? " FIRST" : " LAST"; + sort_children.emplace_back(make_uniq(Value(modifier))); + } + aggr.order_bys.reset(); + + ErrorData error; + auto sort_key = binder.BindScalarFunction(DEFAULT_SCHEMA, "create_sort_key", std::move(sort_children), error); + if (!sort_key) { + error.Throw(); + } + + auto &children = aggr.children; + children.emplace_back(std::move(sort_key)); + + // Look up the arg_xxx_name function in the catalog + QueryErrorContext error_context; + auto &func = Catalog::GetEntry(context, SYSTEM_CATALOG, DEFAULT_SCHEMA, arg_xxx_name, + error_context); + D_ASSERT(func.type == CatalogType::AGGREGATE_FUNCTION_ENTRY); + + // bind the aggregate + vector types; + for (const auto &child : children) { + types.emplace_back(child->return_type); + } + auto best_function = binder.BindFunction(func.name, func.functions, types, error); + if (best_function == DConstants::INVALID_INDEX) { + error.Throw(); + } + // found a matching function! + auto bound_function = func.functions.GetFunctionByOffset(best_function); + return binder.BindAggregateFunction(bound_function, std::move(children), std::move(aggr.filter), + aggr.IsDistinct() ? AggregateType::DISTINCT : AggregateType::NON_DISTINCT); + return nullptr; } diff --git a/src/duckdb/src/optimizer/rule/regex_optimizations.cpp b/src/duckdb/src/optimizer/rule/regex_optimizations.cpp index 00f332ea8..5aa4401c3 100644 --- a/src/duckdb/src/optimizer/rule/regex_optimizations.cpp +++ b/src/duckdb/src/optimizer/rule/regex_optimizations.cpp @@ -5,6 +5,7 @@ #include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/function/scalar/string_functions.hpp" #include "duckdb/function/scalar/regexp.hpp" +#include "utf8proc_wrapper.hpp" #include "re2/re2.h" #include "re2/regexp.h" @@ -39,6 +40,19 @@ static void AddCharacter(char chr, LikeString &ret, bool contains) { ret.like_string += run_as_str; } +static void AddCodepoint(int32_t codepoint, LikeString &ret, bool contains) { + int sz = 0; + char utf8_str[4]; + if (!Utf8Proc::CodepointToUtf8(codepoint, sz, utf8_str)) { + // invalid codepoint + ret.exists = false; + return; + } + for (idx_t i = 0; i < idx_t(sz); i++) { + AddCharacter(utf8_str[i], ret, contains); + } +} + static LikeString GetLikeStringEscaped(duckdb_re2::Regexp *regexp, bool contains = false) { D_ASSERT(regexp->op() == duckdb_re2::kRegexpLiteralString || regexp->op() == duckdb_re2::kRegexpLiteral); LikeString ret; @@ -57,16 +71,14 @@ static LikeString GetLikeStringEscaped(duckdb_re2::Regexp *regexp, bool contains auto nrunes = (idx_t)regexp->nrunes(); auto runes = regexp->runes(); for (idx_t i = 0; i < nrunes; i++) { - char chr = toascii(runes[i]); - AddCharacter(chr, ret, contains); + AddCodepoint(runes[i], ret, contains); if (!ret.exists) { return ret; } } } else { auto rune = regexp->rune(); - char chr = toascii(rune); - AddCharacter(chr, ret, contains); + AddCodepoint(rune, ret, contains); } D_ASSERT(ret.like_string.size() >= 1 || !ret.exists); return ret; diff --git a/src/duckdb/src/optimizer/statistics/expression/propagate_comparison.cpp b/src/duckdb/src/optimizer/statistics/expression/propagate_comparison.cpp index 0a073c546..1331d8167 100644 --- a/src/duckdb/src/optimizer/statistics/expression/propagate_comparison.cpp +++ b/src/duckdb/src/optimizer/statistics/expression/propagate_comparison.cpp @@ -14,6 +14,7 @@ FilterPropagateResult StatisticsPropagator::PropagateComparison(BaseStatistics & case PhysicalType::UINT16: case PhysicalType::UINT32: case PhysicalType::UINT64: + case PhysicalType::UINT128: case PhysicalType::INT8: case PhysicalType::INT16: case PhysicalType::INT32: diff --git a/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp b/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp index 07baf9d5b..5fb8092cd 100644 --- a/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp +++ b/src/duckdb/src/optimizer/statistics/operator/propagate_join.cpp @@ -47,12 +47,8 @@ void StatisticsPropagator::PropagateStatistics(LogicalComparisonJoin &join, uniq if (join.join_type == JoinType::RIGHT_ANTI) { std::swap(join.children[0], join.children[1]); } - // when the right child has data, return the left child - // when the right child has no data, return an empty set - auto limit = make_uniq(1, 0, nullptr, nullptr); - limit->AddChild(std::move(join.children[1])); - auto cross_product = LogicalCrossProduct::Create(std::move(join.children[0]), std::move(limit)); - *node_ptr = std::move(cross_product); + // If the filter is always false or Null, just return the left child. + *node_ptr = std::move(join.children[0]); return; } case JoinType::LEFT: @@ -102,8 +98,11 @@ void StatisticsPropagator::PropagateStatistics(LogicalComparisonJoin &join, uniq if (join.join_type == JoinType::RIGHT_SEMI) { std::swap(join.children[0], join.children[1]); } - // when the right child has data, return the left child // when the right child has no data, return an empty set + // cannot just return the left child because if the right child has no cardinality + // then the whole result should be empty. + // TODO: write better CE logic for limits so that we can just look at + // join.children[1].estimated_cardinality. auto limit = make_uniq(1, 0, nullptr, nullptr); limit->AddChild(std::move(join.children[1])); auto cross_product = LogicalCrossProduct::Create(std::move(join.children[0]), std::move(limit)); @@ -117,6 +116,11 @@ void StatisticsPropagator::PropagateStatistics(LogicalComparisonJoin &join, uniq *node_ptr = std::move(cross_product); return; } + case JoinType::ANTI: + case JoinType::RIGHT_ANTI: { + ReplaceWithEmptyResult(*node_ptr); + return; + } default: // we don't handle mark/single join here yet break; diff --git a/src/duckdb/src/optimizer/unnest_rewriter.cpp b/src/duckdb/src/optimizer/unnest_rewriter.cpp index 4c9350589..f5d91061f 100644 --- a/src/duckdb/src/optimizer/unnest_rewriter.cpp +++ b/src/duckdb/src/optimizer/unnest_rewriter.cpp @@ -85,12 +85,14 @@ void UnnestRewriter::FindCandidates(unique_ptr *op_ptr, } // LHS child is a window - if (delim_join.children[0]->type != LogicalOperatorType::LOGICAL_WINDOW) { + idx_t delim_idx = delim_join.delim_flipped ? 1 : 0; + idx_t other_idx = 1 - delim_idx; + if (delim_join.children[delim_idx]->type != LogicalOperatorType::LOGICAL_WINDOW) { return; } // RHS child must be projection(s) followed by an UNNEST - auto curr_op = &delim_join.children[1]; + auto curr_op = &delim_join.children[other_idx]; while (curr_op->get()->type == LogicalOperatorType::LOGICAL_PROJECTION) { if (curr_op->get()->children.size() != 1) { break; @@ -116,20 +118,23 @@ bool UnnestRewriter::RewriteCandidate(unique_ptr *candidate) { // get the LOGICAL_DELIM_JOIN, which is a child of the candidate D_ASSERT(topmost_op.children.size() == 1); - auto &delim_join = *(topmost_op.children[0]); + auto &delim_join = topmost_op.children[0]->Cast(); D_ASSERT(delim_join.type == LogicalOperatorType::LOGICAL_DELIM_JOIN); GetDelimColumns(delim_join); // LHS of the LOGICAL_DELIM_JOIN is a LOGICAL_WINDOW that contains a LOGICAL_PROJECTION // this lhs_proj later becomes the child of the UNNEST - auto &window = *delim_join.children[0]; + + idx_t delim_idx = delim_join.delim_flipped ? 1 : 0; + idx_t other_idx = 1 - delim_idx; + auto &window = *delim_join.children[delim_idx]; auto &lhs_op = window.children[0]; GetLHSExpressions(*lhs_op); // find the LOGICAL_UNNEST // and get the path down to the LOGICAL_UNNEST vector *> path_to_unnest; - auto curr_op = &(delim_join.children[1]); + auto curr_op = &delim_join.children[other_idx]; while (curr_op->get()->type == LogicalOperatorType::LOGICAL_PROJECTION) { path_to_unnest.push_back(curr_op); curr_op = &curr_op->get()->children[0]; @@ -160,7 +165,7 @@ void UnnestRewriter::UpdateRHSBindings(unique_ptr *plan_ptr, un idx_t shift = lhs_bindings.size(); vector *> path_to_unnest; - auto curr_op = &(topmost_op.children[0]); + auto curr_op = &topmost_op.children[0]; while (curr_op->get()->type == LogicalOperatorType::LOGICAL_PROJECTION) { path_to_unnest.push_back(curr_op); @@ -253,7 +258,7 @@ void UnnestRewriter::UpdateBoundUnnestBindings(UnnestRewriterPlanUpdater &update auto &topmost_op = (LogicalOperator &)**candidate; // traverse LOGICAL_PROJECTION(s) - auto curr_op = &(topmost_op.children[0]); + auto curr_op = &topmost_op.children[0]; while (curr_op->get()->type == LogicalOperatorType::LOGICAL_PROJECTION) { curr_op = &curr_op->get()->children[0]; } diff --git a/src/duckdb/src/parallel/event.cpp b/src/duckdb/src/parallel/event.cpp index 0f51b41f7..0ecfddea5 100644 --- a/src/duckdb/src/parallel/event.cpp +++ b/src/duckdb/src/parallel/event.cpp @@ -44,11 +44,11 @@ void Event::AddDependency(Event &event) { total_dependencies++; event.parents.push_back(weak_ptr(shared_from_this())); #ifdef DEBUG - event.parents_raw.push_back(this); + event.parents_raw.push_back(*this); #endif } -const vector &Event::GetParentsVerification() const { +const vector> &Event::GetParentsVerification() const { D_ASSERT(parents.size() == parents_raw.size()); return parents_raw; } diff --git a/src/duckdb/src/parallel/executor.cpp b/src/duckdb/src/parallel/executor.cpp index 35d23c8d2..eed8e535d 100644 --- a/src/duckdb/src/parallel/executor.cpp +++ b/src/duckdb/src/parallel/executor.cpp @@ -2,8 +2,8 @@ #include "duckdb/execution/execution_context.hpp" #include "duckdb/execution/operator/helper/physical_result_collector.hpp" -#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" #include "duckdb/execution/operator/set/physical_cte.hpp" +#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" #include "duckdb/execution/physical_operator.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/client_data.hpp" @@ -97,7 +97,7 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // create events/stack for this pipeline auto pipeline_event = make_shared(pipeline); - auto finish_group = meta_pipeline->GetFinishGroup(pipeline.get()); + auto finish_group = meta_pipeline->GetFinishGroup(*pipeline); if (finish_group) { // this pipeline is part of a finish group const auto group_entry = event_map.find(*finish_group.get()); @@ -112,7 +112,7 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // add pipeline stack to event map event_map.insert(make_pair(reference(*pipeline), pipeline_stack)); - } else if (meta_pipeline->HasFinishEvent(pipeline.get())) { + } else if (meta_pipeline->HasFinishEvent(*pipeline)) { // this pipeline has its own finish event (despite going into the same sink - Finalize twice!) auto pipeline_finish_event = make_shared(pipeline); PipelineEventStack pipeline_stack(base_stack.pipeline_initialize_event, *pipeline_event, @@ -126,7 +126,6 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S // add pipeline stack to event map event_map.insert(make_pair(reference(*pipeline), pipeline_stack)); - } else { // no additional finish event PipelineEventStack pipeline_stack(base_stack.pipeline_initialize_event, *pipeline_event, @@ -154,15 +153,16 @@ void Executor::SchedulePipeline(const shared_ptr &meta_pipeline, S pipeline->ResetSource(true); } - auto dependencies = meta_pipeline->GetDependencies(pipeline.get()); + auto dependencies = meta_pipeline->GetDependencies(*pipeline); if (!dependencies) { continue; } auto root_entry = event_map.find(*pipeline); D_ASSERT(root_entry != event_map.end()); auto &pipeline_stack = root_entry->second; - for (auto &dependency : *dependencies) { - auto event_entry = event_map.find(*dependency); + // iterate in reverse so the deepest dependencies are added first + for (auto it = dependencies->rbegin(); it != dependencies->rend(); ++it) { + auto event_entry = event_map.find(*it); D_ASSERT(event_entry != event_map.end()); auto &dependency_stack = event_entry->second; pipeline_stack.pipeline_event.AddDependency(dependency_stack.pipeline_event); @@ -212,10 +212,10 @@ void Executor::ScheduleEvents(const vector> &meta_pipel void Executor::VerifyScheduledEvents(const ScheduleEventData &event_data) { #ifdef DEBUG const idx_t count = event_data.events.size(); - vector vertices; + vector> vertices; vertices.reserve(count); for (const auto &event : event_data.events) { - vertices.push_back(event.get()); + vertices.push_back(*event); } vector visited(count, false); vector recursion_stack(count, false); @@ -225,14 +225,14 @@ void Executor::VerifyScheduledEvents(const ScheduleEventData &event_data) { #endif } -void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector &vertices, vector &visited, - vector &recursion_stack) { +void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector> &vertices, + vector &visited, vector &recursion_stack) { D_ASSERT(!recursion_stack[vertex]); // this vertex is in the recursion stack: circular dependency! if (visited[vertex]) { return; // early out: we already visited this vertex } - auto &parents = vertices[vertex]->GetParentsVerification(); + auto &parents = vertices[vertex].get().GetParentsVerification(); if (parents.empty()) { return; // early out: outgoing edges } @@ -243,7 +243,7 @@ void Executor::VerifyScheduledEventsInternal(const idx_t vertex, const vector Executor::GetTypes() { return physical_plan->GetTypes(); } -void Executor::PushError(PreservedError exception) { - // interrupt execution of any other pipelines that belong to this executor - context.interrupted = true; +void Executor::PushError(ErrorData exception) { // push the exception onto the stack error_manager.PushError(std::move(exception)); + // interrupt execution of any other pipelines that belong to this executor + context.interrupted = true; } bool Executor::HasError() { @@ -600,7 +600,7 @@ bool Executor::GetPipelinesProgress(double ¤t_progress, uint64_t ¤t_ D_ASSERT(progress[i] <= 100); current_cardinality += double(progress[i]) * double(cardinality[i]) / double(100); current_progress += progress[i] * double(cardinality[i]) / double(total_cardinality); - D_ASSERT(current_cardinality < total_cardinality); + D_ASSERT(current_cardinality <= total_cardinality); } return true; } // LCOV_EXCL_STOP diff --git a/src/duckdb/src/parallel/executor_task.cpp b/src/duckdb/src/parallel/executor_task.cpp index dc1f0448b..ba0ea1b18 100644 --- a/src/duckdb/src/parallel/executor_task.cpp +++ b/src/duckdb/src/parallel/executor_task.cpp @@ -26,12 +26,10 @@ void ExecutorTask::Reschedule() { TaskExecutionResult ExecutorTask::Execute(TaskExecutionMode mode) { try { return ExecuteTask(mode); - } catch (Exception &ex) { - executor.PushError(PreservedError(ex)); } catch (std::exception &ex) { - executor.PushError(PreservedError(ex)); + executor.PushError(ErrorData(ex)); } catch (...) { // LCOV_EXCL_START - executor.PushError(PreservedError("Unknown exception in Finalize!")); + executor.PushError(ErrorData("Unknown exception in Finalize!")); } // LCOV_EXCL_STOP return TaskExecutionResult::TASK_ERROR; } diff --git a/src/duckdb/src/parallel/meta_pipeline.cpp b/src/duckdb/src/parallel/meta_pipeline.cpp index 191f36c71..ded1cb246 100644 --- a/src/duckdb/src/parallel/meta_pipeline.cpp +++ b/src/duckdb/src/parallel/meta_pipeline.cpp @@ -1,11 +1,10 @@ #include "duckdb/parallel/meta_pipeline.hpp" #include "duckdb/execution/executor.hpp" -#include "duckdb/execution/operator/set/physical_recursive_cte.hpp" namespace duckdb { -MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, PhysicalOperator *sink_p) +MetaPipeline::MetaPipeline(Executor &executor_p, PipelineBuildState &state_p, optional_ptr sink_p) : executor(executor_p), state(state_p), sink(sink_p), recursive_cte(false), next_batch_index(0) { CreatePipeline(); } @@ -46,7 +45,7 @@ void MetaPipeline::GetMetaPipelines(vector> &result, bo } } -const vector *MetaPipeline::GetDependencies(Pipeline *dependant) const { +optional_ptr>> MetaPipeline::GetDependencies(Pipeline &dependant) const { auto it = dependencies.find(dependant); if (it == dependencies.end()) { return nullptr; @@ -63,8 +62,8 @@ void MetaPipeline::SetRecursiveCTE() { recursive_cte = true; } -void MetaPipeline::AssignNextBatchIndex(Pipeline *pipeline) { - pipeline->base_batch_index = next_batch_index++ * PipelineBuildState::BATCH_INCREMENT; +void MetaPipeline::AssignNextBatchIndex(Pipeline &pipeline) { + pipeline.base_batch_index = next_batch_index++ * PipelineBuildState::BATCH_INCREMENT; } void MetaPipeline::Build(PhysicalOperator &op) { @@ -92,16 +91,16 @@ MetaPipeline &MetaPipeline::CreateChildMetaPipeline(Pipeline ¤t, PhysicalO return *child_meta_pipeline; } -Pipeline *MetaPipeline::CreatePipeline() { +Pipeline &MetaPipeline::CreatePipeline() { pipelines.emplace_back(make_shared(executor)); state.SetPipelineSink(*pipelines.back(), sink, next_batch_index++); - return pipelines.back().get(); + return *pipelines.back(); } -void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, bool including) { +void MetaPipeline::AddDependenciesFrom(Pipeline &dependant, Pipeline &start, bool including) { // find 'start' auto it = pipelines.begin(); - for (; it->get() != start; it++) { + for (; !RefersToSameObject(**it, start); it++) { } if (!including) { @@ -109,13 +108,13 @@ void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, boo } // collect pipelines that were created from then - vector created_pipelines; + vector> created_pipelines; for (; it != pipelines.end(); it++) { - if (it->get() == dependant) { + if (RefersToSameObject(**it, dependant)) { // cannot depend on itself continue; } - created_pipelines.push_back(it->get()); + created_pipelines.push_back(**it); } // add them to the dependencies @@ -123,62 +122,62 @@ void MetaPipeline::AddDependenciesFrom(Pipeline *dependant, Pipeline *start, boo deps.insert(deps.begin(), created_pipelines.begin(), created_pipelines.end()); } -void MetaPipeline::AddFinishEvent(Pipeline *pipeline) { +void MetaPipeline::AddFinishEvent(Pipeline &pipeline) { D_ASSERT(finish_pipelines.find(pipeline) == finish_pipelines.end()); finish_pipelines.insert(pipeline); // add all pipelines that were added since 'pipeline' was added (including 'pipeline') to the finish group auto it = pipelines.begin(); - for (; it->get() != pipeline; it++) { + for (; !RefersToSameObject(**it, pipeline); it++) { } it++; for (; it != pipelines.end(); it++) { - finish_map.emplace(it->get(), pipeline); + finish_map.emplace(**it, pipeline); } } -bool MetaPipeline::HasFinishEvent(Pipeline *pipeline) const { +bool MetaPipeline::HasFinishEvent(Pipeline &pipeline) const { return finish_pipelines.find(pipeline) != finish_pipelines.end(); } -optional_ptr MetaPipeline::GetFinishGroup(Pipeline *pipeline) const { +optional_ptr MetaPipeline::GetFinishGroup(Pipeline &pipeline) const { auto it = finish_map.find(pipeline); - return it == finish_map.end() ? nullptr : it->second; + return it == finish_map.end() ? nullptr : &it->second; } -Pipeline *MetaPipeline::CreateUnionPipeline(Pipeline ¤t, bool order_matters) { +Pipeline &MetaPipeline::CreateUnionPipeline(Pipeline ¤t, bool order_matters) { // create the union pipeline (batch index 0, should be set correctly afterwards) - auto union_pipeline = CreatePipeline(); - state.SetPipelineOperators(*union_pipeline, state.GetPipelineOperators(current)); - state.SetPipelineSink(*union_pipeline, sink, 0); + auto &union_pipeline = CreatePipeline(); + state.SetPipelineOperators(union_pipeline, state.GetPipelineOperators(current)); + state.SetPipelineSink(union_pipeline, sink, 0); // 'union_pipeline' inherits ALL dependencies of 'current' (within this MetaPipeline, and across MetaPipelines) - union_pipeline->dependencies = current.dependencies; - auto current_deps = GetDependencies(¤t); + union_pipeline.dependencies = current.dependencies; + auto current_deps = GetDependencies(current); if (current_deps) { dependencies[union_pipeline] = *current_deps; } if (order_matters) { // if we need to preserve order, or if the sink is not parallel, we set a dependency - dependencies[union_pipeline].push_back(¤t); + dependencies[union_pipeline].push_back(current); } return union_pipeline; } -void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator &op, Pipeline *last_pipeline) { +void MetaPipeline::CreateChildPipeline(Pipeline ¤t, PhysicalOperator &op, Pipeline &last_pipeline) { // rule 2: 'current' must be fully built (down to the source) before creating the child pipeline D_ASSERT(current.source); // create the child pipeline (same batch index) pipelines.emplace_back(state.CreateChildPipeline(executor, current, op)); - auto child_pipeline = pipelines.back().get(); - child_pipeline->base_batch_index = current.base_batch_index; + auto &child_pipeline = *pipelines.back(); + child_pipeline.base_batch_index = current.base_batch_index; // child pipeline has a dependency (within this MetaPipeline on all pipelines that were scheduled // between 'current' and now (including 'current') - set them up - dependencies[child_pipeline].push_back(¤t); + dependencies[child_pipeline].push_back(current); AddDependenciesFrom(child_pipeline, last_pipeline, false); D_ASSERT(!GetDependencies(child_pipeline)->empty()); } diff --git a/src/duckdb/src/parallel/pipeline_event.cpp b/src/duckdb/src/parallel/pipeline_event.cpp index 735f9fad3..aaa84a3a2 100644 --- a/src/duckdb/src/parallel/pipeline_event.cpp +++ b/src/duckdb/src/parallel/pipeline_event.cpp @@ -12,12 +12,10 @@ void PipelineEvent::Schedule() { try { pipeline->Schedule(event); D_ASSERT(total_tasks > 0); - } catch (Exception &ex) { - executor.PushError(PreservedError(ex)); } catch (std::exception &ex) { - executor.PushError(PreservedError(ex)); + executor.PushError(ErrorData(ex)); } catch (...) { // LCOV_EXCL_START - executor.PushError(PreservedError("Unknown exception in Finalize!")); + executor.PushError(ErrorData("Unknown exception in Finalize!")); } // LCOV_EXCL_STOP } diff --git a/src/duckdb/src/parallel/pipeline_executor.cpp b/src/duckdb/src/parallel/pipeline_executor.cpp index dff99062f..2c00aef94 100644 --- a/src/duckdb/src/parallel/pipeline_executor.cpp +++ b/src/duckdb/src/parallel/pipeline_executor.cpp @@ -394,12 +394,7 @@ void PipelineExecutor::ExecutePull(DataChunk &result) { } } } - } catch (const Exception &ex) { // LCOV_EXCL_START - if (executor.HasError()) { - executor.ThrowException(); - } - throw; - } catch (std::exception &ex) { + } catch (std::exception &ex) { // LCOV_EXCL_START if (executor.HasError()) { executor.ThrowException(); } diff --git a/src/duckdb/src/parser/column_definition.cpp b/src/duckdb/src/parser/column_definition.cpp index 713618efe..e73fdb025 100644 --- a/src/duckdb/src/parser/column_definition.cpp +++ b/src/duckdb/src/parser/column_definition.cpp @@ -22,6 +22,7 @@ ColumnDefinition ColumnDefinition::Copy() const { copy.expression = expression ? expression->Copy() : nullptr; copy.compression_type = compression_type; copy.category = category; + copy.comment = comment; return copy; } @@ -64,11 +65,18 @@ void ColumnDefinition::SetType(const LogicalType &type) { const string &ColumnDefinition::Name() const { return name; } - void ColumnDefinition::SetName(const string &name) { this->name = name; } +const Value &ColumnDefinition::Comment() const { + return comment; +} + +void ColumnDefinition::SetComment(const Value &comment) { + this->comment = comment; +} + const duckdb::CompressionType &ColumnDefinition::CompressionType() const { return compression_type; } diff --git a/src/duckdb/src/parser/column_list.cpp b/src/duckdb/src/parser/column_list.cpp index 3fe951fee..0f94cbc6d 100644 --- a/src/duckdb/src/parser/column_list.cpp +++ b/src/duckdb/src/parser/column_list.cpp @@ -1,6 +1,7 @@ #include "duckdb/parser/column_list.hpp" #include "duckdb/common/string.hpp" #include "duckdb/common/to_string.hpp" +#include "duckdb/common/exception/catalog_exception.hpp" namespace duckdb { diff --git a/src/duckdb/src/parser/expression/lambda_expression.cpp b/src/duckdb/src/parser/expression/lambda_expression.cpp index 3b9b1398c..be5034114 100644 --- a/src/duckdb/src/parser/expression/lambda_expression.cpp +++ b/src/duckdb/src/parser/expression/lambda_expression.cpp @@ -1,6 +1,8 @@ #include "duckdb/parser/expression/lambda_expression.hpp" + #include "duckdb/common/types/hash.hpp" #include "duckdb/common/string_util.hpp" +#include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" @@ -14,6 +16,55 @@ LambdaExpression::LambdaExpression(unique_ptr lhs, unique_ptr< : ParsedExpression(ExpressionType::LAMBDA, ExpressionClass::LAMBDA), lhs(std::move(lhs)), expr(std::move(expr)) { } +vector> LambdaExpression::ExtractColumnRefExpressions(string &error_message) { + + // we return an error message because we can't throw a binder exception here, + // since we can't distinguish between a lambda function and the JSON operator yet + vector> column_refs; + + if (lhs->expression_class == ExpressionClass::COLUMN_REF) { + // single column reference + column_refs.emplace_back(*lhs); + return column_refs; + } + + if (lhs->expression_class == ExpressionClass::FUNCTION) { + // list of column references + auto &func_expr = lhs->Cast(); + if (func_expr.function_name != "row") { + error_message = InvalidParametersErrorMessage(); + return column_refs; + } + + for (auto &child : func_expr.children) { + if (child->expression_class != ExpressionClass::COLUMN_REF) { + error_message = InvalidParametersErrorMessage(); + return column_refs; + } + column_refs.emplace_back(*child); + } + } + + if (column_refs.empty()) { + error_message = InvalidParametersErrorMessage(); + } + return column_refs; +} + +string LambdaExpression::InvalidParametersErrorMessage() { + return "Invalid lambda parameters! Parameters must be unqualified comma-separated names like x or (x, y)."; +} + +bool LambdaExpression::IsLambdaParameter(const vector> &lambda_params, + const string ¶meter_name) { + for (const auto &level : lambda_params) { + if (level.find(parameter_name) != level.end()) { + return true; + } + } + return false; +} + string LambdaExpression::ToString() const { return "(" + lhs->ToString() + " -> " + expr->ToString() + ")"; } diff --git a/src/duckdb/src/parser/expression/lambdaref_expression.cpp b/src/duckdb/src/parser/expression/lambdaref_expression.cpp index 461e4a348..786d10912 100644 --- a/src/duckdb/src/parser/expression/lambdaref_expression.cpp +++ b/src/duckdb/src/parser/expression/lambdaref_expression.cpp @@ -1,6 +1,7 @@ #include "duckdb/parser/expression/lambdaref_expression.hpp" #include "duckdb/common/types/hash.hpp" +#include "duckdb/planner/table_binding.hpp" namespace duckdb { @@ -33,4 +34,26 @@ unique_ptr LambdaRefExpression::Copy() const { throw InternalException("lambda reference expressions are transient, Copy should never be called"); } +unique_ptr +LambdaRefExpression::FindMatchingBinding(optional_ptr> &lambda_bindings, + const string &column_name) { + + // if this is a lambda parameter, then we temporarily add a BoundLambdaRef, + // which we capture and remove later + + // inner lambda parameters have precedence over outer lambda parameters, and + // lambda parameters have precedence over macros and columns + + if (lambda_bindings) { + for (idx_t i = lambda_bindings->size(); i > 0; i--) { + if ((*lambda_bindings)[i - 1].HasMatchingBinding(column_name)) { + D_ASSERT(!(*lambda_bindings)[i - 1].alias.empty()); + return make_uniq(i - 1, column_name); + } + } + } + + return nullptr; +} + } // namespace duckdb diff --git a/src/duckdb/src/parser/expression/window_expression.cpp b/src/duckdb/src/parser/expression/window_expression.cpp index ca1990d30..66e7fe835 100644 --- a/src/duckdb/src/parser/expression/window_expression.cpp +++ b/src/duckdb/src/parser/expression/window_expression.cpp @@ -14,7 +14,7 @@ WindowExpression::WindowExpression(ExpressionType type) : ParsedExpression(type, WindowExpression::WindowExpression(ExpressionType type, string catalog_name, string schema, const string &function_name) : ParsedExpression(type, ExpressionClass::WINDOW), catalog(std::move(catalog_name)), schema(std::move(schema)), - function_name(StringUtil::Lower(function_name)), ignore_nulls(false) { + function_name(StringUtil::Lower(function_name)), ignore_nulls(false), distinct(false) { switch (type) { case ExpressionType::WINDOW_AGGREGATE: case ExpressionType::WINDOW_ROW_NUMBER: @@ -70,6 +70,9 @@ bool WindowExpression::Equal(const WindowExpression &a, const WindowExpression & if (a.ignore_nulls != b.ignore_nulls) { return false; } + if (a.distinct != b.distinct) { + return false; + } if (!ParsedExpression::ListEquals(a.children, b.children)) { return false; } @@ -136,6 +139,7 @@ unique_ptr WindowExpression::Copy() const { new_window->offset_expr = offset_expr ? offset_expr->Copy() : nullptr; new_window->default_expr = default_expr ? default_expr->Copy() : nullptr; new_window->ignore_nulls = ignore_nulls; + new_window->distinct = distinct; return std::move(new_window); } diff --git a/src/duckdb/src/parser/parsed_data/alter_table_info.cpp b/src/duckdb/src/parser/parsed_data/alter_table_info.cpp index 29ef6e90e..0f9967024 100644 --- a/src/duckdb/src/parser/parsed_data/alter_table_info.cpp +++ b/src/duckdb/src/parser/parsed_data/alter_table_info.cpp @@ -25,6 +25,28 @@ unique_ptr ChangeOwnershipInfo::Copy() const { owner_name, if_not_found); } +//===--------------------------------------------------------------------===// +// SetCommentInfo +//===--------------------------------------------------------------------===// +SetCommentInfo::SetCommentInfo(CatalogType entry_catalog_type, string entry_catalog_p, string entry_schema_p, + string entry_name_p, Value new_comment_value_p, OnEntryNotFound if_not_found) + : AlterInfo(AlterType::SET_COMMENT, std::move(entry_catalog_p), std::move(entry_schema_p), std::move(entry_name_p), + if_not_found), + entry_catalog_type(entry_catalog_type), comment_value(std::move(new_comment_value_p)) { +} + +CatalogType SetCommentInfo::GetCatalogType() const { + return entry_catalog_type; +} + +unique_ptr SetCommentInfo::Copy() const { + return make_uniq_base(entry_catalog_type, catalog, schema, name, comment_value, + if_not_found); +} + +SetCommentInfo::SetCommentInfo() : AlterInfo(AlterType::SET_COMMENT) { +} + //===--------------------------------------------------------------------===// // AlterTableInfo //===--------------------------------------------------------------------===// @@ -132,6 +154,24 @@ unique_ptr ChangeColumnTypeInfo::Copy() const { expression->Copy()); } +//===--------------------------------------------------------------------===// +// SetColumnCommentInfo +//===--------------------------------------------------------------------===// +SetColumnCommentInfo::SetColumnCommentInfo() : AlterTableInfo(AlterTableType::SET_COLUMN_COMMENT) { +} + +SetColumnCommentInfo::SetColumnCommentInfo(AlterEntryData data, string column_name, Value comment_value) + : AlterTableInfo(AlterTableType::SET_COLUMN_COMMENT, std::move(data)), column_name(std::move(column_name)), + comment(std::move(comment_value)) { +} + +SetColumnCommentInfo::~SetColumnCommentInfo() { +} + +unique_ptr SetColumnCommentInfo::Copy() const { + return make_uniq_base(GetAlterEntryData(), column_name, comment); +} + //===--------------------------------------------------------------------===// // SetDefaultInfo //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/parser/parsed_data/comment_on_info.cpp b/src/duckdb/src/parser/parsed_data/comment_on_info.cpp new file mode 100644 index 000000000..2be18857f --- /dev/null +++ b/src/duckdb/src/parser/parsed_data/comment_on_info.cpp @@ -0,0 +1,19 @@ +#include "duckdb/parser/parsed_data/comment_on_info.hpp" + +namespace duckdb { + +CommentOnInfo::CommentOnInfo() + : ParseInfo(TYPE), catalog(INVALID_CATALOG), schema(INVALID_SCHEMA), name(""), comment(Value()) { +} + +unique_ptr CommentOnInfo::Copy() const { + auto result = make_uniq(); + result->type = type; + result->catalog = catalog; + result->schema = schema; + result->name = name; + result->comment = comment; + return result; +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_data/create_info.cpp b/src/duckdb/src/parser/parsed_data/create_info.cpp index cd5ad2dc6..a548955b3 100644 --- a/src/duckdb/src/parser/parsed_data/create_info.cpp +++ b/src/duckdb/src/parser/parsed_data/create_info.cpp @@ -19,6 +19,7 @@ void CreateInfo::CopyProperties(CreateInfo &other) const { other.temporary = temporary; other.internal = internal; other.sql = sql; + other.comment = comment; } unique_ptr CreateInfo::GetAlterInfo() const { diff --git a/src/duckdb/src/parser/parsed_data/create_secret_info.cpp b/src/duckdb/src/parser/parsed_data/create_secret_info.cpp new file mode 100644 index 000000000..55f8bd020 --- /dev/null +++ b/src/duckdb/src/parser/parsed_data/create_secret_info.cpp @@ -0,0 +1,22 @@ +#include "duckdb/parser/parsed_data/create_secret_info.hpp" + +#include "duckdb/parser/parsed_data/create_info.hpp" + +namespace duckdb { + +CreateSecretInfo::CreateSecretInfo(OnCreateConflict on_conflict, SecretPersistType persist_type) + : CreateInfo(CatalogType::SECRET_ENTRY), on_conflict(on_conflict), persist_type(persist_type), options() { +} + +unique_ptr CreateSecretInfo::Copy() const { + auto result = make_uniq(on_conflict, persist_type); + result->type = type; + result->storage_type = storage_type; + result->provider = provider; + result->name = name; + result->scope = scope; + result->options = options; + return std::move(result); +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_data/create_view_info.cpp b/src/duckdb/src/parser/parsed_data/create_view_info.cpp index 792f2547b..4f30137f4 100644 --- a/src/duckdb/src/parser/parsed_data/create_view_info.cpp +++ b/src/duckdb/src/parser/parsed_data/create_view_info.cpp @@ -55,21 +55,25 @@ unique_ptr CreateViewInfo::Copy() const { return std::move(result); } -unique_ptr CreateViewInfo::FromSelect(ClientContext &context, unique_ptr info) { - D_ASSERT(info); - D_ASSERT(!info->view_name.empty()); - D_ASSERT(!info->sql.empty()); - D_ASSERT(!info->query); - +unique_ptr CreateViewInfo::ParseSelect(const string &sql) { Parser parser; - parser.ParseQuery(info->sql); + parser.ParseQuery(sql); if (parser.statements.size() != 1 || parser.statements[0]->type != StatementType::SELECT_STATEMENT) { throw BinderException( "Failed to create view from SQL string - \"%s\" - statement did not contain a single SELECT statement", - info->sql); + sql); } D_ASSERT(parser.statements.size() == 1 && parser.statements[0]->type == StatementType::SELECT_STATEMENT); - info->query = unique_ptr_cast(std::move(parser.statements[0])); + return unique_ptr_cast(std::move(parser.statements[0])); +} + +unique_ptr CreateViewInfo::FromSelect(ClientContext &context, unique_ptr info) { + D_ASSERT(info); + D_ASSERT(!info->view_name.empty()); + D_ASSERT(!info->sql.empty()); + D_ASSERT(!info->query); + + info->query = ParseSelect(info->sql); auto binder = Binder::CreateBinder(context); binder->BindCreateViewInfo(*info); diff --git a/src/duckdb/src/parser/parsed_data/drop_info.cpp b/src/duckdb/src/parser/parsed_data/drop_info.cpp index 9c0b57c4c..ce8567504 100644 --- a/src/duckdb/src/parser/parsed_data/drop_info.cpp +++ b/src/duckdb/src/parser/parsed_data/drop_info.cpp @@ -1,20 +1,19 @@ #include "duckdb/parser/parsed_data/drop_info.hpp" +#include "duckdb/parser/parsed_data/extra_drop_info.hpp" namespace duckdb { DropInfo::DropInfo() : ParseInfo(TYPE), catalog(INVALID_CATALOG), schema(INVALID_SCHEMA), cascade(false) { } +DropInfo::DropInfo(const DropInfo &info) + : ParseInfo(info.info_type), type(info.type), catalog(info.catalog), schema(info.schema), name(info.name), + if_not_found(info.if_not_found), cascade(info.cascade), allow_drop_internal(info.allow_drop_internal), + extra_drop_info(info.extra_drop_info ? info.extra_drop_info->Copy() : nullptr) { +} + unique_ptr DropInfo::Copy() const { - auto result = make_uniq(); - result->type = type; - result->catalog = catalog; - result->schema = schema; - result->name = name; - result->if_not_found = if_not_found; - result->cascade = cascade; - result->allow_drop_internal = allow_drop_internal; - return result; + return make_uniq(*this); } } // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_data/extra_drop_info.cpp b/src/duckdb/src/parser/parsed_data/extra_drop_info.cpp new file mode 100644 index 000000000..b120e653f --- /dev/null +++ b/src/duckdb/src/parser/parsed_data/extra_drop_info.cpp @@ -0,0 +1,16 @@ +#include "duckdb/parser/parsed_data/extra_drop_info.hpp" + +namespace duckdb { + +ExtraDropSecretInfo::ExtraDropSecretInfo() : ExtraDropInfo(ExtraDropInfoType::SECRET_INFO) { +} + +ExtraDropSecretInfo::ExtraDropSecretInfo(const ExtraDropSecretInfo &info) + : ExtraDropInfo(ExtraDropInfoType::SECRET_INFO) { +} + +unique_ptr ExtraDropSecretInfo::Copy() const { + return std::move(make_uniq(*this)); +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/parsed_expression_iterator.cpp b/src/duckdb/src/parser/parsed_expression_iterator.cpp index 77d60316a..50a466917 100644 --- a/src/duckdb/src/parser/parsed_expression_iterator.cpp +++ b/src/duckdb/src/parser/parsed_expression_iterator.cpp @@ -240,7 +240,8 @@ void ParsedExpressionIterator::EnumerateTableRefChildren( break; } case TableReferenceType::BASE_TABLE: - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: + case TableReferenceType::SHOW_REF: // these TableRefs do not need to be unfolded break; case TableReferenceType::INVALID: diff --git a/src/duckdb/src/parser/parser.cpp b/src/duckdb/src/parser/parser.cpp index 597db79d8..9d2cf7559 100644 --- a/src/duckdb/src/parser/parser.cpp +++ b/src/duckdb/src/parser/parser.cpp @@ -150,6 +150,7 @@ vector SplitQueryStringIntoStatements(const string &query) { void Parser::ParseQuery(const string &query) { Transformer transformer(options); string parser_error; + optional_idx parser_error_location; { // check if there are any unicode spaces in the string string new_query; @@ -178,7 +179,10 @@ void Parser::ParseQuery(const string &query) { transformer.TransformParseTree(parser.parse_tree, statements); parsing_succeed = true; } else { - parser_error = QueryErrorContext::Format(query, parser.error_message, parser.error_location - 1); + parser_error = parser.error_message; + if (parser.error_location > 0) { + parser_error_location = parser.error_location - 1; + } } } // If DuckDB fails to parse the entire sql string, break the string down into individual statements @@ -188,13 +192,13 @@ void Parser::ParseQuery(const string &query) { // return here would require refactoring into another function. o.w. will just no-op in order to run wrap up // code at the end of this function } else if (!options.extensions || options.extensions->empty()) { - throw ParserException(parser_error); + throw ParserException::SyntaxError(query, parser_error, parser_error_location); } else { // split sql string into statements and re-parse using extension auto query_statements = SplitQueryStringIntoStatements(query); auto stmt_loc = 0; for (auto const &query_statement : query_statements) { - string another_parser_error; + ErrorData another_parser_error; // Creating a new scope to allow extensions to use PostgresParser, which is not reentrant { PostgresParser another_parser; @@ -213,8 +217,10 @@ void Parser::ParseQuery(const string &query) { stmt_loc += query_statement.size(); continue; } else { - another_parser_error = QueryErrorContext::Format(query, another_parser.error_message, - another_parser.error_location - 1); + another_parser_error = ErrorData(another_parser.error_message); + if (another_parser.error_location > 0) { + another_parser_error.AddQueryLocation(another_parser.error_location - 1); + } } } // LCOV_EXCL_STOP // LCOV_EXCL_START @@ -233,13 +239,13 @@ void Parser::ParseQuery(const string &query) { parsed_single_statement = true; break; } else if (result.type == ParserExtensionResultType::DISPLAY_EXTENSION_ERROR) { - throw ParserException(result.error); + throw ParserException::SyntaxError(query, result.error, result.error_location); } else { // We move to the next one! } } if (!parsed_single_statement) { - throw ParserException(parser_error); + throw ParserException::SyntaxError(query, parser_error, parser_error_location); } // LCOV_EXCL_STOP } } diff --git a/src/duckdb/src/parser/query_error_context.cpp b/src/duckdb/src/parser/query_error_context.cpp index 58f3418b6..65f3eaff7 100644 --- a/src/duckdb/src/parser/query_error_context.cpp +++ b/src/duckdb/src/parser/query_error_context.cpp @@ -7,12 +7,17 @@ namespace duckdb { -string QueryErrorContext::Format(const string &query, const string &error_message, int error_loc) { - if (error_loc < 0 || size_t(error_loc) >= query.size()) { +string QueryErrorContext::Format(const string &query, const string &error_message, optional_idx error_loc, + bool add_line_indicator) { + if (!error_loc.IsValid()) { // no location in query provided return error_message; } - idx_t error_location = idx_t(error_loc); + idx_t error_location = error_loc.GetIndex(); + if (error_location >= query.size()) { + // out of bounds + return error_message; + } // count the line numbers until the error location // and set the start position as the first character of that line idx_t start_pos = 0; @@ -90,7 +95,10 @@ string QueryErrorContext::Format(const string &query, const string &error_messag break; } } - string line_indicator = "LINE " + to_string(line_number) + ": "; + string line_indicator; + if (add_line_indicator) { + line_indicator = "LINE " + to_string(line_number) + ": "; + } string begin_trunc = truncate_beginning ? "..." : ""; string end_trunc = truncate_end ? "..." : ""; @@ -109,13 +117,4 @@ string QueryErrorContext::Format(const string &query, const string &error_messag return result; } -string QueryErrorContext::FormatErrorRecursive(const string &msg, vector &values) { - string error_message = values.empty() ? msg : ExceptionFormatValue::Format(msg, values); - if (!statement || query_location >= statement->query.size()) { - // no statement provided or query location out of range - return error_message; - } - return Format(statement->query, error_message, query_location); -} - } // namespace duckdb diff --git a/src/duckdb/src/parser/query_node/select_node.cpp b/src/duckdb/src/parser/query_node/select_node.cpp index e228bc398..66ad5dc72 100644 --- a/src/duckdb/src/parser/query_node/select_node.cpp +++ b/src/duckdb/src/parser/query_node/select_node.cpp @@ -11,6 +11,10 @@ SelectNode::SelectNode() } string SelectNode::ToString() const { + if (from_table && from_table->type == TableReferenceType::SHOW_REF) { + D_ASSERT(select_list.size() == 1); + return from_table->ToString(); + } string result; result = cte_map.ToString(); result += "SELECT "; @@ -41,7 +45,7 @@ string SelectNode::ToString() const { result += StringUtil::Format(" AS %s", SQLIdentifier(select_list[i]->alias)); } } - if (from_table && from_table->type != TableReferenceType::EMPTY) { + if (from_table && from_table->type != TableReferenceType::EMPTY_FROM) { result += " FROM " + from_table->ToString(); } if (where_clause) { diff --git a/src/duckdb/src/parser/statement/show_statement.cpp b/src/duckdb/src/parser/statement/show_statement.cpp deleted file mode 100644 index e7abb4fda..000000000 --- a/src/duckdb/src/parser/statement/show_statement.cpp +++ /dev/null @@ -1,15 +0,0 @@ -#include "duckdb/parser/statement/show_statement.hpp" - -namespace duckdb { - -ShowStatement::ShowStatement() : SQLStatement(StatementType::SHOW_STATEMENT), info(make_uniq()) { -} - -ShowStatement::ShowStatement(const ShowStatement &other) : SQLStatement(other), info(other.info->Copy()) { -} - -unique_ptr ShowStatement::Copy() const { - return unique_ptr(new ShowStatement(*this)); -} - -} // namespace duckdb diff --git a/src/duckdb/src/parser/tableref/basetableref.cpp b/src/duckdb/src/parser/tableref/basetableref.cpp index a90a6f383..a39c07ec7 100644 --- a/src/duckdb/src/parser/tableref/basetableref.cpp +++ b/src/duckdb/src/parser/tableref/basetableref.cpp @@ -34,4 +34,5 @@ unique_ptr BaseTableRef::Copy() { return std::move(copy); } + } // namespace duckdb diff --git a/src/duckdb/src/parser/tableref/showref.cpp b/src/duckdb/src/parser/tableref/showref.cpp new file mode 100644 index 000000000..a403d6e83 --- /dev/null +++ b/src/duckdb/src/parser/tableref/showref.cpp @@ -0,0 +1,47 @@ +#include "duckdb/parser/tableref/showref.hpp" + +namespace duckdb { + +ShowRef::ShowRef() : TableRef(TableReferenceType::SHOW_REF), show_type(ShowType::DESCRIBE) { +} + +string ShowRef::ToString() const { + string result; + if (show_type == ShowType::SUMMARY) { + result += "SUMMARIZE "; + } else { + result += "DESCRIBE "; + } + if (query) { + result += query->ToString(); + } else if (table_name != "__show_tables_expanded") { + result += table_name; + } + return result; +} + +bool ShowRef::Equals(const TableRef &other_p) const { + if (!TableRef::Equals(other_p)) { + return false; + } + auto &other = other_p.Cast(); + if (other.query.get() != query.get()) { + if (!other.query->Equals(query.get())) { + return false; + } + } + return table_name == other.table_name && show_type == other.show_type; +} + +unique_ptr ShowRef::Copy() { + auto copy = make_uniq(); + + copy->table_name = table_name; + copy->query = query ? query->Copy() : nullptr; + copy->show_type = show_type; + CopyProperties(*copy); + + return std::move(copy); +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/transform/expression/transform_bool_expr.cpp b/src/duckdb/src/parser/transform/expression/transform_bool_expr.cpp index b7de69fab..40a98ccb6 100644 --- a/src/duckdb/src/parser/transform/expression/transform_bool_expr.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_bool_expr.cpp @@ -46,6 +46,7 @@ unique_ptr Transformer::TransformBoolExpr(duckdb_libpgquery::P } } } + SetQueryLocation(*result, root.location); return result; } diff --git a/src/duckdb/src/parser/transform/expression/transform_boolean_test.cpp b/src/duckdb/src/parser/transform/expression/transform_boolean_test.cpp index 78ee75aba..9ee96a931 100644 --- a/src/duckdb/src/parser/transform/expression/transform_boolean_test.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_boolean_test.cpp @@ -6,31 +6,46 @@ namespace duckdb { +static unique_ptr TransformBooleanTestInternal(unique_ptr argument, + ExpressionType comparison_type, bool comparison_value, + int query_location) { + auto bool_value = make_uniq(Value::BOOLEAN(comparison_value)); + Transformer::SetQueryLocation(*bool_value, query_location); + // we cast the argument to bool to remove ambiguity wrt function binding on the comparision + auto cast_argument = make_uniq(LogicalType::BOOLEAN, std::move(argument)); + + auto result = make_uniq(comparison_type, std::move(cast_argument), std::move(bool_value)); + Transformer::SetQueryLocation(*result, query_location); + return std::move(result); +} + +static unique_ptr TransformBooleanTestIsNull(unique_ptr argument, + ExpressionType operator_type, idx_t query_location) { + auto result = make_uniq(operator_type, std::move(argument)); + Transformer::SetQueryLocation(*result, query_location); + return std::move(result); +} + unique_ptr Transformer::TransformBooleanTest(duckdb_libpgquery::PGBooleanTest &node) { auto argument = TransformExpression(PGPointerCast(node.arg)); - auto expr_true = make_uniq(Value::BOOLEAN(true)); - auto expr_false = make_uniq(Value::BOOLEAN(false)); - // we cast the argument to bool to remove ambiguity wrt function binding on the comparision - auto cast_argument = make_uniq(LogicalType::BOOLEAN, argument->Copy()); - switch (node.booltesttype) { case duckdb_libpgquery::PGBoolTestType::PG_IS_TRUE: - return make_uniq(ExpressionType::COMPARE_NOT_DISTINCT_FROM, std::move(cast_argument), - std::move(expr_true)); + return TransformBooleanTestInternal(std::move(argument), ExpressionType::COMPARE_NOT_DISTINCT_FROM, true, + node.location); case duckdb_libpgquery::PGBoolTestType::IS_NOT_TRUE: - return make_uniq(ExpressionType::COMPARE_DISTINCT_FROM, std::move(cast_argument), - std::move(expr_true)); + return TransformBooleanTestInternal(std::move(argument), ExpressionType::COMPARE_DISTINCT_FROM, true, + node.location); case duckdb_libpgquery::PGBoolTestType::IS_FALSE: - return make_uniq(ExpressionType::COMPARE_NOT_DISTINCT_FROM, std::move(cast_argument), - std::move(expr_false)); + return TransformBooleanTestInternal(std::move(argument), ExpressionType::COMPARE_NOT_DISTINCT_FROM, false, + node.location); case duckdb_libpgquery::PGBoolTestType::IS_NOT_FALSE: - return make_uniq(ExpressionType::COMPARE_DISTINCT_FROM, std::move(cast_argument), - std::move(expr_false)); + return TransformBooleanTestInternal(std::move(argument), ExpressionType::COMPARE_DISTINCT_FROM, false, + node.location); case duckdb_libpgquery::PGBoolTestType::IS_UNKNOWN: // IS NULL - return make_uniq(ExpressionType::OPERATOR_IS_NULL, std::move(argument)); + return TransformBooleanTestIsNull(std::move(argument), ExpressionType::OPERATOR_IS_NULL, node.location); case duckdb_libpgquery::PGBoolTestType::IS_NOT_UNKNOWN: // IS NOT NULL - return make_uniq(ExpressionType::OPERATOR_IS_NOT_NULL, std::move(argument)); + return TransformBooleanTestIsNull(std::move(argument), ExpressionType::OPERATOR_IS_NOT_NULL, node.location); default: throw NotImplementedException("Unknown boolean test type %d", node.booltesttype); } diff --git a/src/duckdb/src/parser/transform/expression/transform_case.cpp b/src/duckdb/src/parser/transform/expression/transform_case.cpp index 2902fa436..73eeb3189 100644 --- a/src/duckdb/src/parser/transform/expression/transform_case.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_case.cpp @@ -29,6 +29,7 @@ unique_ptr Transformer::TransformCase(duckdb_libpgquery::PGCas } else { case_node->else_expr = make_uniq(Value(LogicalType::SQLNULL)); } + SetQueryLocation(*case_node, root.location); return std::move(case_node); } diff --git a/src/duckdb/src/parser/transform/expression/transform_columnref.cpp b/src/duckdb/src/parser/transform/expression/transform_columnref.cpp index 7cfb0d7ea..b93ff1a30 100644 --- a/src/duckdb/src/parser/transform/expression/transform_columnref.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_columnref.cpp @@ -58,7 +58,7 @@ unique_ptr Transformer::TransformStarExpression(duckdb_libpgqu } } result->columns = star.columns; - result->query_location = star.location; + SetQueryLocation(*result, star.location); return std::move(result); } @@ -75,7 +75,7 @@ unique_ptr Transformer::TransformColumnRef(duckdb_libpgquery:: column_names.emplace_back(PGPointerCast(node->data.ptr_value)->val.str); } auto colref = make_uniq(std::move(column_names)); - colref->query_location = root.location; + SetQueryLocation(*colref, root.location); return std::move(colref); } case duckdb_libpgquery::T_PGAStar: { diff --git a/src/duckdb/src/parser/transform/expression/transform_constant.cpp b/src/duckdb/src/parser/transform/expression/transform_constant.cpp index 39f021e02..c64d47d6b 100644 --- a/src/duckdb/src/parser/transform/expression/transform_constant.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_constant.cpp @@ -22,6 +22,8 @@ unique_ptr Transformer::TransformValue(duckdb_libpgquery::PG bool try_cast_as_integer = true; bool try_cast_as_decimal = true; int decimal_position = -1; + int num_underscores = 0; + int num_integer_underscores = 0; for (idx_t i = 0; i < str_val.GetSize(); i++) { if (val.val.str[i] == '.') { // decimal point: cast as either decimal or double @@ -33,6 +35,12 @@ unique_ptr Transformer::TransformValue(duckdb_libpgquery::PG try_cast_as_integer = false; try_cast_as_decimal = false; } + if (val.val.str[i] == '_') { + num_underscores++; + if (decimal_position < 0) { + num_integer_underscores++; + } + } } if (try_cast_as_integer) { int64_t bigint_value; @@ -50,10 +58,10 @@ unique_ptr Transformer::TransformValue(duckdb_libpgquery::PG } idx_t decimal_offset = val.val.str[0] == '-' ? 3 : 2; if (try_cast_as_decimal && decimal_position >= 0 && - str_val.GetSize() < Decimal::MAX_WIDTH_DECIMAL + decimal_offset) { + str_val.GetSize() - num_underscores < Decimal::MAX_WIDTH_DECIMAL + decimal_offset) { // figure out the width/scale based on the decimal position - auto width = uint8_t(str_val.GetSize() - 1); - auto scale = uint8_t(width - decimal_position); + auto width = uint8_t(str_val.GetSize() - 1 - num_underscores); + auto scale = uint8_t(width - decimal_position + num_integer_underscores); if (val.val.str[0] == '-') { width--; } @@ -76,7 +84,9 @@ unique_ptr Transformer::TransformValue(duckdb_libpgquery::PG } unique_ptr Transformer::TransformConstant(duckdb_libpgquery::PGAConst &c) { - return TransformValue(c.val); + auto constant = TransformValue(c.val); + SetQueryLocation(*constant, c.location); + return std::move(constant); } bool Transformer::ConstructConstantFromExpression(const ParsedExpression &expr, Value &value) { @@ -112,9 +122,9 @@ bool Transformer::ConstructConstantFromExpression(const ParsedExpression &expr, } // figure out child type - LogicalType child_type(LogicalTypeId::INTEGER); + LogicalType child_type(LogicalTypeId::SQLNULL); for (auto &child_value : values) { - child_type = LogicalType::MaxLogicalType(child_type, child_value.type()); + child_type = LogicalType::ForceMaxLogicalType(child_type, child_value.type()); } // finally create the list diff --git a/src/duckdb/src/parser/transform/expression/transform_function.cpp b/src/duckdb/src/parser/transform/expression/transform_function.cpp index c51bd35e9..606477fd8 100644 --- a/src/duckdb/src/parser/transform/expression/transform_function.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_function.cpp @@ -148,8 +148,8 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P throw InternalException("Unknown/unsupported window function"); } - if (root.agg_distinct) { - throw ParserException("DISTINCT is not implemented for window functions!"); + if (win_fun_type != ExpressionType::WINDOW_AGGREGATE && root.agg_distinct) { + throw ParserException("DISTINCT is not implemented for non-aggregate window functions!"); } if (root.agg_order) { @@ -169,6 +169,7 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P auto expr = make_uniq(win_fun_type, std::move(catalog), std::move(schema), lowercase_name); expr->ignore_nulls = root.agg_ignore_nulls; + expr->distinct = root.agg_distinct; if (root.agg_filter) { auto filter_expr = TransformExpression(root.agg_filter); @@ -206,7 +207,7 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P } auto window_spec = PGPointerCast(root.over); if (window_spec->name) { - auto it = window_clauses.find(StringUtil::Lower(string(window_spec->name))); + auto it = window_clauses.find(string(window_spec->name)); if (it == window_clauses.end()) { throw ParserException("window \"%s\" does not exist", window_spec->name); } @@ -216,7 +217,7 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P auto window_ref = window_spec; auto window_name = window_ref->refname; if (window_ref->refname) { - auto it = window_clauses.find(StringUtil::Lower(string(window_spec->refname))); + auto it = window_clauses.find(string(window_spec->refname)); if (it == window_clauses.end()) { throw ParserException("window \"%s\" does not exist", window_spec->refname); } @@ -233,7 +234,7 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P } TransformWindowFrame(*window_spec, *expr); in_window_definition = false; - expr->query_location = root.location; + SetQueryLocation(*expr, root.location); return std::move(expr); } @@ -335,7 +336,7 @@ unique_ptr Transformer::TransformFuncCall(duckdb_libpgquery::P auto function = make_uniq(std::move(catalog), std::move(schema), lowercase_name.c_str(), std::move(children), std::move(filter_expr), std::move(order_bys), root.agg_distinct, false, root.export_state); - function->query_location = root.location; + SetQueryLocation(*function, root.location); return std::move(function); } diff --git a/src/duckdb/src/parser/transform/expression/transform_grouping_function.cpp b/src/duckdb/src/parser/transform/expression/transform_grouping_function.cpp index 36751a9e5..a3300cb80 100644 --- a/src/duckdb/src/parser/transform/expression/transform_grouping_function.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_grouping_function.cpp @@ -9,7 +9,7 @@ unique_ptr Transformer::TransformGroupingFunction(duckdb_libpg auto n = PGPointerCast(node->data.ptr_value); op->children.push_back(TransformExpression(n)); } - op->query_location = grouping.location; + SetQueryLocation(*op, grouping.location); return std::move(op); } diff --git a/src/duckdb/src/parser/transform/expression/transform_interval.cpp b/src/duckdb/src/parser/transform/expression/transform_interval.cpp index 90e749c2f..9865bb03e 100644 --- a/src/duckdb/src/parser/transform/expression/transform_interval.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_interval.cpp @@ -55,6 +55,7 @@ unique_ptr Transformer::TransformInterval(duckdb_libpgquery::P // (we might add support if someone complains about it) string fname; + LogicalType parse_type = LogicalType::DOUBLE; LogicalType target_type; if (mask & YEAR_MASK && mask & MONTH_MASK) { // DAY TO HOUR @@ -92,11 +93,11 @@ unique_ptr Transformer::TransformInterval(duckdb_libpgquery::P } else if (mask & HOUR_MASK) { // HOUR fname = "to_hours"; - target_type = LogicalType::DOUBLE; + target_type = LogicalType::BIGINT; } else if (mask & MINUTE_MASK) { // MINUTE fname = "to_minutes"; - target_type = LogicalType::DOUBLE; + target_type = LogicalType::BIGINT; } else if (mask & SECOND_MASK) { // SECOND fname = "to_seconds"; @@ -128,8 +129,16 @@ unique_ptr Transformer::TransformInterval(duckdb_libpgquery::P } else { throw InternalException("Unsupported interval post-fix"); } - // first push a cast to the target type - expr = make_uniq(target_type, std::move(expr)); + // first push a cast to the parse type + expr = make_uniq(parse_type, std::move(expr)); + + // next, truncate it if the target type doesn't match the parse type + if (target_type != parse_type) { + vector> children; + children.push_back(std::move(expr)); + expr = make_uniq("trunc", std::move(children)); + expr = make_uniq(target_type, std::move(expr)); + } // now push the operation vector> children; children.push_back(std::move(expr)); diff --git a/src/duckdb/src/parser/transform/expression/transform_is_null.cpp b/src/duckdb/src/parser/transform/expression/transform_is_null.cpp index a6bd014c2..c588438b0 100644 --- a/src/duckdb/src/parser/transform/expression/transform_is_null.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_is_null.cpp @@ -13,7 +13,9 @@ unique_ptr Transformer::TransformNullTest(duckdb_libpgquery::P ? ExpressionType::OPERATOR_IS_NULL : ExpressionType::OPERATOR_IS_NOT_NULL; - return unique_ptr(new OperatorExpression(expr_type, std::move(arg))); + auto result = make_uniq(expr_type, std::move(arg)); + SetQueryLocation(*result, root.location); + return std::move(result); } } // namespace duckdb diff --git a/src/duckdb/src/parser/transform/expression/transform_lambda.cpp b/src/duckdb/src/parser/transform/expression/transform_lambda.cpp index 4f980a12d..00011ad33 100644 --- a/src/duckdb/src/parser/transform/expression/transform_lambda.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_lambda.cpp @@ -12,7 +12,9 @@ unique_ptr Transformer::TransformLambda(duckdb_libpgquery::PGL auto rhs = TransformExpression(node.rhs); D_ASSERT(lhs); D_ASSERT(rhs); - return make_uniq(std::move(lhs), std::move(rhs)); + auto result = make_uniq(std::move(lhs), std::move(rhs)); + SetQueryLocation(*result, node.location); + return std::move(result); } } // namespace duckdb diff --git a/src/duckdb/src/parser/transform/expression/transform_operator.cpp b/src/duckdb/src/parser/transform/expression/transform_operator.cpp index 826a3558f..3c3a87f0e 100644 --- a/src/duckdb/src/parser/transform/expression/transform_operator.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_operator.cpp @@ -81,7 +81,7 @@ unique_ptr Transformer::TransformAExprInternal(duckdb_libpgque subquery_expr->subquery_type = SubqueryType::ANY; subquery_expr->child = std::move(left_expr); subquery_expr->comparison_type = OperatorToExpressionType(name); - subquery_expr->query_location = root.location; + SetQueryLocation(*subquery_expr, root.location); if (subquery_expr->comparison_type == ExpressionType::INVALID) { throw ParserException("Unsupported comparison \"%s\" for ANY/ALL subquery", name); } @@ -107,7 +107,7 @@ unique_ptr Transformer::TransformAExprInternal(duckdb_libpgque operator_type = ExpressionType::COMPARE_IN; } auto result = make_uniq(operator_type, std::move(left_expr)); - result->query_location = root.location; + SetQueryLocation(*result, root.location); TransformExpressionList(*PGPointerCast(root.rexpr), result->children); return std::move(result); } @@ -210,7 +210,7 @@ unique_ptr Transformer::TransformAExprInternal(duckdb_libpgque unique_ptr Transformer::TransformAExpr(duckdb_libpgquery::PGAExpr &root) { auto result = TransformAExprInternal(root); if (result) { - result->query_location = root.location; + SetQueryLocation(*result, root.location); } return result; } diff --git a/src/duckdb/src/parser/transform/expression/transform_positional_reference.cpp b/src/duckdb/src/parser/transform/expression/transform_positional_reference.cpp index 55cb64724..efe7d3da3 100644 --- a/src/duckdb/src/parser/transform/expression/transform_positional_reference.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_positional_reference.cpp @@ -9,7 +9,7 @@ unique_ptr Transformer::TransformPositionalReference(duckdb_li throw ParserException("Positional reference node needs to be >= 1"); } auto result = make_uniq(node.position); - result->query_location = node.location; + SetQueryLocation(*result, node.location); return std::move(result); } diff --git a/src/duckdb/src/parser/transform/expression/transform_subquery.cpp b/src/duckdb/src/parser/transform/expression/transform_subquery.cpp index c4ab20a8a..5b6368900 100644 --- a/src/duckdb/src/parser/transform/expression/transform_subquery.cpp +++ b/src/duckdb/src/parser/transform/expression/transform_subquery.cpp @@ -9,8 +9,9 @@ unique_ptr Transformer::TransformSubquery(duckdb_libpgquery::P auto subquery_expr = make_uniq(); subquery_expr->subquery = TransformSelect(root.subselect); + SetQueryLocation(*subquery_expr, root.location); D_ASSERT(subquery_expr->subquery); - D_ASSERT(subquery_expr->subquery->node->GetSelectList().size() > 0); + D_ASSERT(!subquery_expr->subquery->node->GetSelectList().empty()); switch (root.subLinkType) { case duckdb_libpgquery::PG_EXISTS_SUBLINK: { @@ -55,19 +56,20 @@ unique_ptr Transformer::TransformSubquery(duckdb_libpgquery::P break; } case duckdb_libpgquery::PG_ARRAY_SUBLINK: { - auto subquery_table_alias = "__subquery"; - auto subquery_column_alias = "__arr_element"; - // ARRAY expression - // wrap subquery into "SELECT CASE WHEN ARRAY_AGG(i) IS NULL THEN [] ELSE ARRAY_AGG(i) END FROM (...) tbl(i)" + // wrap subquery into + // "SELECT CASE WHEN ARRAY_AGG(COLUMNS(*)) IS NULL THEN [] ELSE ARRAY_AGG(COLUMNS(*)) END FROM (...) tbl" auto select_node = make_uniq(); - // ARRAY_AGG(i) + // COLUMNS(*) + auto columns_star = make_uniq(); + columns_star->columns = true; + + // ARRAY_AGG(COLUMNS(*)) vector> children; - children.push_back( - make_uniq_base(subquery_column_alias, subquery_table_alias)); + children.push_back(std::move(columns_star)); auto aggr = make_uniq("array_agg", std::move(children)); - // ARRAY_AGG(i) IS NULL + // ARRAY_AGG(COLUMNS(*)) IS NULL auto agg_is_null = make_uniq(ExpressionType::OPERATOR_IS_NULL, aggr->Copy()); // empty list vector> list_children; @@ -82,9 +84,8 @@ unique_ptr Transformer::TransformSubquery(duckdb_libpgquery::P select_node->select_list.push_back(std::move(case_expr)); - // FROM (...) tbl(i) - auto child_subquery = make_uniq(std::move(subquery_expr->subquery), subquery_table_alias); - child_subquery->column_name_alias.emplace_back(subquery_column_alias); + // FROM (...) tbl + auto child_subquery = make_uniq(std::move(subquery_expr->subquery)); select_node->from_table = std::move(child_subquery); auto new_subquery = make_uniq(); @@ -97,7 +98,6 @@ unique_ptr Transformer::TransformSubquery(duckdb_libpgquery::P default: throw NotImplementedException("Subquery of type %d not implemented\n", (int)root.subLinkType); } - subquery_expr->query_location = root.location; return std::move(subquery_expr); } diff --git a/src/duckdb/src/parser/transform/helpers/nodetype_to_string.cpp b/src/duckdb/src/parser/transform/helpers/nodetype_to_string.cpp index 99081e2e7..145632866 100644 --- a/src/duckdb/src/parser/transform/helpers/nodetype_to_string.cpp +++ b/src/duckdb/src/parser/transform/helpers/nodetype_to_string.cpp @@ -478,6 +478,8 @@ std::string Transformer::NodetypeToString(duckdb_libpgquery::PGNodeTag type) { / return "T_ClosePortalStmt"; case duckdb_libpgquery::T_PGClusterStmt: return "T_ClusterStmt"; + case duckdb_libpgquery::T_PGCommentOnStmt: + return "T_CommentOnStmt"; case duckdb_libpgquery::T_PGCopyStmt: return "T_CopyStmt"; case duckdb_libpgquery::T_PGCreateStmt: diff --git a/src/duckdb/src/parser/transform/helpers/transform_cte.cpp b/src/duckdb/src/parser/transform/helpers/transform_cte.cpp index dded0d7d6..ca9192412 100644 --- a/src/duckdb/src/parser/transform/helpers/transform_cte.cpp +++ b/src/duckdb/src/parser/transform/helpers/transform_cte.cpp @@ -33,9 +33,9 @@ void Transformer::ExtractCTEsRecursive(CommonTableExpressionMap &cte_map) { void Transformer::TransformCTE(duckdb_libpgquery::PGWithClause &de_with_clause, CommonTableExpressionMap &cte_map, vector> &materialized_ctes) { - // TODO: might need to update in case of future lawsuit stored_cte_map.push_back(&cte_map); + // TODO: might need to update in case of future lawsuit D_ASSERT(de_with_clause.ctes); for (auto cte_ele = de_with_clause.ctes->head; cte_ele != nullptr; cte_ele = cte_ele->next) { auto info = make_uniq(); diff --git a/src/duckdb/src/parser/transform/statement/transform_comment_on.cpp b/src/duckdb/src/parser/transform/statement/transform_comment_on.cpp new file mode 100644 index 000000000..d48e4fa1f --- /dev/null +++ b/src/duckdb/src/parser/transform/statement/transform_comment_on.cpp @@ -0,0 +1,108 @@ +#include "duckdb/parser/expression/constant_expression.hpp" +#include "duckdb/parser/parsed_data/alter_info.hpp" +#include "duckdb/parser/parsed_data/alter_table_info.hpp" +#include "duckdb/parser/statement/alter_statement.hpp" +#include "duckdb/parser/transformer.hpp" + +namespace duckdb { + +unique_ptr Transformer::TransformCommentOn(duckdb_libpgquery::PGCommentOnStmt &stmt) { + QualifiedName qualified_name; + string column_name; + + if (stmt.object_type != duckdb_libpgquery::PG_OBJECT_COLUMN) { + qualified_name = TransformQualifiedName(*stmt.name); + } else { + auto transformed_expr = TransformExpression(stmt.column_expr); + + if (transformed_expr->GetExpressionType() != ExpressionType::COLUMN_REF) { + throw ParserException("Unexpected expression found, expected column reference to comment on (e.g. " + "'schema.table.column'), found '%s'", + transformed_expr->ToString()); + } + + auto colref_expr = transformed_expr->Cast(); + + if (colref_expr.column_names.size() > 4) { + throw ParserException("Invalid column reference: '%s', too many dots", colref_expr.ToString()); + } + if (colref_expr.column_names.size() < 2) { + throw ParserException("Invalid column reference: '%s', please specify a table", colref_expr.ToString()); + } + + column_name = colref_expr.GetColumnName(); + qualified_name.name = colref_expr.column_names.size() > 1 ? colref_expr.GetTableName() : ""; + + if (colref_expr.column_names.size() == 4) { + qualified_name.catalog = colref_expr.column_names[0]; + qualified_name.schema = colref_expr.column_names[1]; + } else if (colref_expr.column_names.size() == 3) { + qualified_name.schema = colref_expr.column_names[0]; + } + } + + auto res = make_uniq(); + unique_ptr info; + + auto expr = TransformExpression(stmt.value); + if (expr->expression_class != ExpressionClass::CONSTANT) { + throw NotImplementedException("Can only use constants as comments"); + } + auto comment_value = expr->Cast().value; + + CatalogType type = CatalogType::INVALID; + + // Regular CatalogTypes + switch (stmt.object_type) { + case duckdb_libpgquery::PG_OBJECT_TABLE: + type = CatalogType::TABLE_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_INDEX: + type = CatalogType::INDEX_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_VIEW: + type = CatalogType::VIEW_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_FUNCTION: + type = CatalogType::MACRO_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_TABLE_MACRO: + type = CatalogType::TABLE_MACRO_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_SEQUENCE: + type = CatalogType::SEQUENCE_ENTRY; + break; + case duckdb_libpgquery::PG_OBJECT_TYPE: + type = CatalogType::TYPE_ENTRY; + break; + default: + break; + } + + if (type != CatalogType::INVALID) { + info = make_uniq(type, qualified_name.catalog, qualified_name.schema, qualified_name.name, + comment_value, OnEntryNotFound::THROW_EXCEPTION); + } else if (stmt.object_type == duckdb_libpgquery::PG_OBJECT_COLUMN) { + // Special case: Table Column + AlterEntryData alter_entry_data; + alter_entry_data.catalog = qualified_name.catalog; + alter_entry_data.schema = qualified_name.schema; + alter_entry_data.name = qualified_name.name; + alter_entry_data.if_not_found = OnEntryNotFound::THROW_EXCEPTION; + + info = make_uniq(alter_entry_data, column_name, comment_value); + } else if (stmt.object_type == duckdb_libpgquery::PG_OBJECT_DATABASE) { + throw NotImplementedException("Adding comments to databases is not implemented"); + } else if (stmt.object_type == duckdb_libpgquery::PG_OBJECT_SCHEMA) { + throw NotImplementedException("Adding comments to schemas is not implemented"); + } + + if (info) { + res->info = std::move(info); + return res; + } + + throw NotImplementedException("Can not comment on this type"); +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/transform/statement/transform_copy.cpp b/src/duckdb/src/parser/transform/statement/transform_copy.cpp index d50948024..909ab168b 100644 --- a/src/duckdb/src/parser/transform/statement/transform_copy.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_copy.cpp @@ -12,13 +12,54 @@ namespace duckdb { +void Transformer::ParseGenericOptionListEntry(case_insensitive_map_t> &result_options, string &name, + duckdb_libpgquery::PGNode *arg) { + // otherwise + if (result_options.find(name) != result_options.end()) { + throw ParserException("Unexpected duplicate option \"%s\"", name); + } + if (!arg) { + result_options[name] = vector(); + return; + } + switch (arg->type) { + case duckdb_libpgquery::T_PGList: { + auto column_list = PGPointerCast(arg); + for (auto c = column_list->head; c != nullptr; c = lnext(c)) { + auto target = PGPointerCast(c->data.ptr_value); + result_options[name].push_back(Value(target->name)); + } + break; + } + case duckdb_libpgquery::T_PGAStar: + result_options[name].push_back(Value("*")); + break; + case duckdb_libpgquery::T_PGFuncCall: { + auto func_call = PGPointerCast(arg); + auto func_expr = TransformFuncCall(*func_call); + + Value value; + if (!Transformer::ConstructConstantFromExpression(*func_expr, value)) { + throw ParserException("Unsupported expression in option list: %s", func_expr->ToString()); + } + result_options[name].push_back(std::move(value)); + break; + } + default: { + auto val = PGPointerCast(arg); + result_options[name].push_back(TransformValue(*val)->value); + break; + } + } +} + void Transformer::TransformCopyOptions(CopyInfo &info, optional_ptr options) { if (!options) { return; } - // iterate over each option duckdb_libpgquery::PGListCell *cell; + // iterate over each option for_each_cell(cell, options->head) { auto def_elem = PGPointerCast(cell->data.ptr_value); if (StringUtil::Lower(def_elem->defname) == "format") { @@ -30,43 +71,10 @@ void Transformer::TransformCopyOptions(CopyInfo &info, optional_ptrval.str); continue; } - // otherwise - if (info.options.find(def_elem->defname) != info.options.end()) { - throw ParserException("Unexpected duplicate option \"%s\"", def_elem->defname); - } - if (!def_elem->arg) { - info.options[def_elem->defname] = vector(); - continue; - } - switch (def_elem->arg->type) { - case duckdb_libpgquery::T_PGList: { - auto column_list = PGPointerCast(def_elem->arg); - for (auto c = column_list->head; c != nullptr; c = lnext(c)) { - auto target = PGPointerCast(c->data.ptr_value); - info.options[def_elem->defname].push_back(Value(target->name)); - } - break; - } - case duckdb_libpgquery::T_PGAStar: - info.options[def_elem->defname].push_back(Value("*")); - break; - case duckdb_libpgquery::T_PGFuncCall: { - auto func_call = PGPointerCast(def_elem->arg); - auto func_expr = TransformFuncCall(*func_call); - Value value; - if (!Transformer::ConstructConstantFromExpression(*func_expr, value)) { - throw ParserException("Unsupported expression in COPY options: %s", func_expr->ToString()); - } - info.options[def_elem->defname].push_back(std::move(value)); - break; - } - default: { - auto val = PGPointerCast(def_elem->arg); - info.options[def_elem->defname].push_back(TransformValue(*val)->value); - break; - } - } + // The rest ends up in the options + string name = def_elem->defname; + ParseGenericOptionListEntry(info.options, name, def_elem->arg); } } diff --git a/src/duckdb/src/parser/transform/statement/transform_delete.cpp b/src/duckdb/src/parser/transform/statement/transform_delete.cpp index 48cde7b4c..bad33043c 100644 --- a/src/duckdb/src/parser/transform/statement/transform_delete.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_delete.cpp @@ -17,7 +17,7 @@ unique_ptr Transformer::TransformDelete(duckdb_libpgquery::PGDe result->condition = TransformExpression(stmt.whereClause); result->table = TransformRangeVar(*stmt.relation); if (result->table->type != TableReferenceType::BASE_TABLE) { - throw Exception("Can only delete from base tables!"); + throw InvalidInputException("Can only delete from base tables!"); } if (stmt.usingClause) { for (auto n = stmt.usingClause->head; n != nullptr; n = n->next) { diff --git a/src/duckdb/src/parser/transform/statement/transform_drop.cpp b/src/duckdb/src/parser/transform/statement/transform_drop.cpp index 7c6fe9b0d..13ebcaa8e 100644 --- a/src/duckdb/src/parser/transform/statement/transform_drop.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_drop.cpp @@ -1,4 +1,5 @@ #include "duckdb/parser/statement/drop_statement.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/parser/transformer.hpp" namespace duckdb { @@ -73,4 +74,28 @@ unique_ptr Transformer::TransformDrop(duckdb_libpgquery::PGDropStm return std::move(result); } +unique_ptr Transformer::TransformDropSecret(duckdb_libpgquery::PGDropSecretStmt &stmt) { + auto result = make_uniq(); + auto info = make_uniq(); + auto extra_info = make_uniq(); + + info->type = CatalogType::SECRET_ENTRY; + info->name = stmt.secret_name; + info->if_not_found = stmt.missing_ok ? OnEntryNotFound::RETURN_NULL : OnEntryNotFound::THROW_EXCEPTION; + + extra_info->persist_mode = EnumUtil::FromString(StringUtil::Upper(stmt.persist_type)); + extra_info->secret_storage = stmt.secret_storage; + + if (extra_info->persist_mode == SecretPersistType::TEMPORARY) { + if (!extra_info->secret_storage.empty()) { + throw ParserException("Can not combine TEMPORARY with specifying a storage for drop secret"); + } + } + + info->extra_drop_info = std::move(extra_info); + result->info = std::move(info); + + return result; +} + } // namespace duckdb diff --git a/src/duckdb/src/parser/transform/statement/transform_secret.cpp b/src/duckdb/src/parser/transform/statement/transform_secret.cpp new file mode 100644 index 000000000..f9f13b088 --- /dev/null +++ b/src/duckdb/src/parser/transform/statement/transform_secret.cpp @@ -0,0 +1,103 @@ +#include "duckdb/parser/statement/create_statement.hpp" +#include "duckdb/parser/expression/constant_expression.hpp" +#include "duckdb/parser/expression/function_expression.hpp" +#include "duckdb/parser/tableref/basetableref.hpp" +#include "duckdb/parser/transformer.hpp" + +namespace duckdb { + +void Transformer::TransformCreateSecretOptions(CreateSecretInfo &info, + optional_ptr options) { + if (!options) { + return; + } + + duckdb_libpgquery::PGListCell *cell; + // iterate over each option + for_each_cell(cell, options->head) { + auto def_elem = PGPointerCast(cell->data.ptr_value); + auto lower_name = StringUtil::Lower(def_elem->defname); + if (lower_name == "scope") { + // format specifier: interpret this option + auto scope_val = PGPointerCast(def_elem->arg); + if (!scope_val) { + throw ParserException("Unsupported parameter type for SCOPE"); + } else if (scope_val->type == duckdb_libpgquery::T_PGString) { + info.scope.push_back(scope_val->val.str); + continue; + } else if (scope_val->type != duckdb_libpgquery::T_PGList) { + throw ParserException("%s has to be a string, or a list of strings", lower_name); + } + + auto list = PGPointerCast(def_elem->arg); + for (auto scope_cell = list->head; scope_cell != nullptr; scope_cell = lnext(scope_cell)) { + auto scope_val_entry = PGPointerCast(scope_cell->data.ptr_value); + info.scope.push_back(scope_val_entry->val.str); + } + continue; + } else if (lower_name == "type") { + auto type_val = PGPointerCast(def_elem->arg); + if (type_val->type != duckdb_libpgquery::T_PGString) { + throw ParserException("%s has to be a string", lower_name); + } + info.type = StringUtil::Lower(type_val->val.str); + continue; + } else if (lower_name == "provider") { + auto provider_val = PGPointerCast(def_elem->arg); + if (provider_val->type != duckdb_libpgquery::T_PGString) { + throw ParserException("%s has to be a string", lower_name); + } + info.provider = StringUtil::Lower(provider_val->val.str); + continue; + } + + // All the other options end up in the generic + case_insensitive_map_t> vector_options; + ParseGenericOptionListEntry(vector_options, lower_name, def_elem->arg); + + for (const auto &entry : vector_options) { + if (entry.second.size() != 1) { + throw ParserException("Invalid parameter passed to option '%s'", entry.first); + } + + if (info.options.find(entry.first) != info.options.end()) { + throw BinderException("Duplicate query param found while parsing create secret: '%s'", entry.first); + } + + info.options[entry.first] = entry.second.at(0); + } + } +} + +unique_ptr Transformer::TransformSecret(duckdb_libpgquery::PGCreateSecretStmt &stmt) { + auto result = make_uniq(); + + auto create_secret_info = + make_uniq(TransformOnConflict(stmt.onconflict), + EnumUtil::FromString(StringUtil::Upper(stmt.persist_type))); + + if (stmt.secret_name) { + create_secret_info->name = StringUtil::Lower(stmt.secret_name); + } + + if (stmt.secret_storage) { + create_secret_info->storage_type = StringUtil::Lower(stmt.secret_storage); + } + + if (stmt.options) { + TransformCreateSecretOptions(*create_secret_info, stmt.options); + } + + if (create_secret_info->type.empty()) { + throw ParserException("Failed to create secret - secret must have a type defined"); + } + if (create_secret_info->name.empty()) { + create_secret_info->name = "__default_" + create_secret_info->type; + } + + result->info = std::move(create_secret_info); + + return result; +} + +} // namespace duckdb diff --git a/src/duckdb/src/parser/transform/statement/transform_select.cpp b/src/duckdb/src/parser/transform/statement/transform_select.cpp index 5473c9c1d..15331a3ab 100644 --- a/src/duckdb/src/parser/transform/statement/transform_select.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_select.cpp @@ -30,7 +30,14 @@ unique_ptr Transformer::TransformSelect(duckdb_libpgquery::PGSe } unique_ptr Transformer::TransformSelect(optional_ptr node, bool is_select) { - return TransformSelect(PGCast(*node), is_select); + switch (node->type) { + case duckdb_libpgquery::T_PGVariableShowSelectStmt: + return TransformShowSelect(PGCast(*node)); + case duckdb_libpgquery::T_PGVariableShowStmt: + return TransformShow(PGCast(*node)); + default: + return TransformSelect(PGCast(*node), is_select); + } } } // namespace duckdb diff --git a/src/duckdb/src/parser/transform/statement/transform_select_node.cpp b/src/duckdb/src/parser/transform/statement/transform_select_node.cpp index 39c823e08..c024476b1 100644 --- a/src/duckdb/src/parser/transform/statement/transform_select_node.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_select_node.cpp @@ -19,6 +19,7 @@ void Transformer::TransformModifiers(duckdb_libpgquery::PGSelectStmt &stmt, Quer order_modifier->orders = std::move(orders); node.modifiers.push_back(std::move(order_modifier)); } + if (stmt.limitCount || stmt.limitOffset) { if (stmt.limitCount && stmt.limitCount->type == duckdb_libpgquery::T_PGLimitPercent) { auto limit_percent_modifier = make_uniq(); @@ -122,7 +123,7 @@ unique_ptr Transformer::TransformSelectInternal(duckdb_libpgquery::PG result.left = TransformSelectNode(*stmt.larg); result.right = TransformSelectNode(*stmt.rarg); if (!result.left || !result.right) { - throw Exception("Failed to transform setop children."); + throw InternalException("Failed to transform setop children."); } result.setop_all = stmt.all; @@ -140,7 +141,7 @@ unique_ptr Transformer::TransformSelectInternal(duckdb_libpgquery::PG result.setop_type = SetOperationType::UNION_BY_NAME; break; default: - throw Exception("Unexpected setop type"); + throw InternalException("Unexpected setop type"); } if (stmt.sampleOptions) { throw ParserException("SAMPLE clause is only allowed in regular SELECT statements"); diff --git a/src/duckdb/src/parser/transform/statement/transform_show.cpp b/src/duckdb/src/parser/transform/statement/transform_show.cpp index 461a563a0..c80ea92a2 100644 --- a/src/duckdb/src/parser/transform/statement/transform_show.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_show.cpp @@ -1,55 +1,27 @@ #include "duckdb/parser/statement/pragma_statement.hpp" #include "duckdb/parser/transformer.hpp" -#include "duckdb/parser/statement/show_statement.hpp" #include "duckdb/parser/query_node/select_node.hpp" #include "duckdb/parser/expression/star_expression.hpp" #include "duckdb/parser/tableref/basetableref.hpp" #include "duckdb/parser/expression/constant_expression.hpp" +#include "duckdb/parser/statement/select_statement.hpp" +#include "duckdb/parser/tableref/showref.hpp" namespace duckdb { -static void TransformShowName(unique_ptr &result, const string &name) { - auto &info = *result->info; - auto lname = StringUtil::Lower(name); +unique_ptr Transformer::TransformShow(duckdb_libpgquery::PGVariableShowStmt &stmt) { + string name = stmt.name; - if (lname == "\"databases\"") { - info.name = "show_databases"; - } else if (lname == "\"tables\"") { - // show all tables - info.name = "show_tables"; - } else if (lname == "__show_tables_expanded") { - info.name = "show_tables_expanded"; - } else { - // show one specific table - info.name = "show"; - info.parameters.emplace_back(make_uniq(Value(name))); - } -} - -unique_ptr Transformer::TransformShow(duckdb_libpgquery::PGVariableShowStmt &stmt) { - // we transform SHOW x into PRAGMA SHOW('x') - if (stmt.is_summary) { - auto result = make_uniq(); - auto &info = *result->info; - info.is_summary = stmt.is_summary; - - auto select = make_uniq(); - select->select_list.push_back(make_uniq()); - auto basetable = make_uniq(); - auto qualified_name = QualifiedName::Parse(stmt.name); - basetable->schema_name = qualified_name.schema; - basetable->table_name = qualified_name.name; - select->from_table = std::move(basetable); - - info.query = std::move(select); - return std::move(result); - } - - auto result = make_uniq(); + auto select_node = make_uniq(); + select_node->select_list.push_back(make_uniq()); + auto showref = make_uniq(); + showref->table_name = std::move(name); + showref->show_type = stmt.is_summary ? ShowType::SUMMARY : ShowType::DESCRIBE; + select_node->from_table = std::move(showref); - auto show_name = stmt.name; - TransformShowName(result, show_name); - return std::move(result); + auto result = make_uniq(); + result->node = std::move(select_node); + return result; } } // namespace duckdb diff --git a/src/duckdb/src/parser/transform/statement/transform_show_select.cpp b/src/duckdb/src/parser/transform/statement/transform_show_select.cpp index 5fee8a7af..c22c94867 100644 --- a/src/duckdb/src/parser/transform/statement/transform_show_select.cpp +++ b/src/duckdb/src/parser/transform/statement/transform_show_select.cpp @@ -1,19 +1,26 @@ -#include "duckdb/parser/statement/show_statement.hpp" #include "duckdb/parser/sql_statement.hpp" #include "duckdb/parser/transformer.hpp" +#include "duckdb/parser/expression/star_expression.hpp" +#include "duckdb/parser/statement/select_statement.hpp" +#include "duckdb/parser/query_node/select_node.hpp" +#include "duckdb/parser/tableref/showref.hpp" namespace duckdb { -unique_ptr Transformer::TransformShowSelect(duckdb_libpgquery::PGVariableShowSelectStmt &stmt) { +unique_ptr Transformer::TransformShowSelect(duckdb_libpgquery::PGVariableShowSelectStmt &stmt) { // we capture the select statement of SHOW auto select_stmt = PGPointerCast(stmt.stmt); - auto result = make_uniq(); - auto &info = *result->info; - info.is_summary = stmt.is_summary; + auto select_node = make_uniq(); + select_node->select_list.push_back(make_uniq()); - info.query = TransformSelectNode(*select_stmt); + auto show_ref = make_uniq(); + show_ref->show_type = stmt.is_summary ? ShowType::SUMMARY : ShowType::DESCRIBE; + show_ref->query = TransformSelectNode(*select_stmt); + select_node->from_table = std::move(show_ref); + auto result = make_uniq(); + result->node = std::move(select_node); return result; } diff --git a/src/duckdb/src/parser/transform/tableref/transform_base_tableref.cpp b/src/duckdb/src/parser/transform/tableref/transform_base_tableref.cpp index 6e6cbf1ae..d0e64a452 100644 --- a/src/duckdb/src/parser/transform/tableref/transform_base_tableref.cpp +++ b/src/duckdb/src/parser/transform/tableref/transform_base_tableref.cpp @@ -19,7 +19,7 @@ unique_ptr Transformer::TransformRangeVar(duckdb_libpgquery::PGRangeVa if (root.sample) { result->sample = TransformSampleOptions(root.sample); } - result->query_location = root.location; + SetQueryLocation(*result, root.location); return std::move(result); } diff --git a/src/duckdb/src/parser/transform/tableref/transform_join.cpp b/src/duckdb/src/parser/transform/tableref/transform_join.cpp index bd7924104..4213cb5a2 100644 --- a/src/duckdb/src/parser/transform/tableref/transform_join.cpp +++ b/src/duckdb/src/parser/transform/tableref/transform_join.cpp @@ -54,8 +54,7 @@ unique_ptr Transformer::TransformJoin(duckdb_libpgquery::PGJoinExpr &r default: break; } - result->query_location = root.location; - + SetQueryLocation(*result, root.location); if (root.usingClause && root.usingClause->length > 0) { // usingClause is a list of strings for (auto node = root.usingClause->head; node != nullptr; node = node->next) { diff --git a/src/duckdb/src/parser/transform/tableref/transform_table_function.cpp b/src/duckdb/src/parser/transform/tableref/transform_table_function.cpp index 5d8b05c2d..43dccc743 100644 --- a/src/duckdb/src/parser/transform/tableref/transform_table_function.cpp +++ b/src/duckdb/src/parser/transform/tableref/transform_table_function.cpp @@ -28,7 +28,7 @@ unique_ptr Transformer::TransformRangeFunction(duckdb_libpgquery::PGRa case duckdb_libpgquery::T_PGFuncCall: { auto func_call = PGPointerCast(call_tree.get()); result->function = TransformFuncCall(*func_call); - result->query_location = func_call->location; + SetQueryLocation(*result, func_call->location); break; } case duckdb_libpgquery::T_PGSQLValueFunction: diff --git a/src/duckdb/src/parser/transformer.cpp b/src/duckdb/src/parser/transformer.cpp index 761736b92..e36d7c2eb 100644 --- a/src/duckdb/src/parser/transformer.cpp +++ b/src/duckdb/src/parser/transformer.cpp @@ -211,6 +211,12 @@ unique_ptr Transformer::TransformStatementInternal(duckdb_libpgque return TransformUse(PGCast(stmt)); case duckdb_libpgquery::T_PGCopyDatabaseStmt: return TransformCopyDatabase(PGCast(stmt)); + case duckdb_libpgquery::T_PGCreateSecretStmt: + return TransformSecret(PGCast(stmt)); + case duckdb_libpgquery::T_PGDropSecretStmt: + return TransformDropSecret(PGCast(stmt)); + case duckdb_libpgquery::T_PGCommentOnStmt: + return TransformCommentOn(PGCast(stmt)); default: throw NotImplementedException(NodetypeToString(stmt.type)); } @@ -230,4 +236,18 @@ unique_ptr Transformer::TransformMaterializedCTE(unique_ptrTryGetBindingIndex(column_name, binding_index)) { // LCOV_EXCL_START @@ -162,10 +162,10 @@ unordered_set BindContext::GetMatchingBindings(const string &column_name } unique_ptr BindContext::ExpandGeneratedColumn(const string &table_name, const string &column_name) { - string error_message; + ErrorData error; - auto binding = GetBinding(table_name, error_message); - D_ASSERT(binding); + auto binding = GetBinding(table_name, error); + D_ASSERT(binding && !error.HasError()); auto &table_binding = binding->Cast(); auto result = table_binding.ExpandGeneratedColumn(column_name); result->alias = column_name; @@ -196,7 +196,7 @@ static bool ColumnIsGenerated(Binding &binding, column_t index) { unique_ptr BindContext::CreateColumnReference(const string &catalog_name, const string &schema_name, const string &table_name, const string &column_name) { - string error_message; + ErrorData error; vector names; if (!catalog_name.empty()) { names.push_back(catalog_name); @@ -208,7 +208,7 @@ unique_ptr BindContext::CreateColumnReference(const string &ca names.push_back(column_name); auto result = make_uniq(std::move(names)); - auto binding = GetBinding(table_name, error_message); + auto binding = GetBinding(table_name, error); if (!binding) { return std::move(result); } @@ -237,7 +237,7 @@ optional_ptr BindContext::GetCTEBinding(const string &ctename) { return match->second.get(); } -optional_ptr BindContext::GetBinding(const string &name, string &out_error) { +optional_ptr BindContext::GetBinding(const string &name, ErrorData &out_error) { auto match = bindings.find(name); if (match == bindings.end()) { // alias not found in this BindContext @@ -247,7 +247,8 @@ optional_ptr BindContext::GetBinding(const string &name, string &out_er } string candidate_str = StringUtil::CandidatesMessage(StringUtil::TopNLevenshtein(candidates, name), "Candidate tables"); - out_error = StringUtil::Format("Referenced table \"%s\" not found!%s", name, candidate_str); + out_error = ErrorData(ExceptionType::BINDER, + StringUtil::Format("Referenced table \"%s\" not found!%s", name, candidate_str)); return nullptr; } return match->second.get(); @@ -258,10 +259,10 @@ BindResult BindContext::BindColumn(ColumnRefExpression &colref, idx_t depth) { throw InternalException("Could not bind alias \"%s\"!", colref.GetColumnName()); } - string error; + ErrorData error; auto binding = GetBinding(colref.GetTableName(), error); if (!binding) { - return BindResult(error); + return BindResult(std::move(error)); } return binding->Bind(colref, depth); } @@ -367,13 +368,13 @@ void BindContext::GenerateAllColumnExpressions(StarExpression &expr, } else { // SELECT tbl.* case // SELECT struct.* case - string error; + ErrorData error; auto binding = GetBinding(expr.relation_name, error); bool is_struct_ref = false; if (!binding) { auto binding_name = GetMatchingBinding(expr.relation_name); if (binding_name.empty()) { - throw BinderException(error); + error.Throw(); } binding = bindings[binding_name].get(); is_struct_ref = true; diff --git a/src/duckdb/src/planner/binder.cpp b/src/duckdb/src/planner/binder.cpp index 4097622db..5a81ee6ec 100644 --- a/src/duckdb/src/planner/binder.cpp +++ b/src/duckdb/src/planner/binder.cpp @@ -96,8 +96,6 @@ BoundStatement Binder::Bind(SQLStatement &statement) { return Bind(statement.Cast()); case StatementType::VACUUM_STATEMENT: return Bind(statement.Cast()); - case StatementType::SHOW_STATEMENT: - return Bind(statement.Cast()); case StatementType::CALL_STATEMENT: return Bind(statement.Cast()); case StatementType::EXPORT_STATEMENT: @@ -194,7 +192,7 @@ unique_ptr Binder::Bind(TableRef &ref) { case TableReferenceType::SUBQUERY: result = Bind(ref.Cast()); break; - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: result = Bind(ref.Cast()); break; case TableReferenceType::TABLE_FUNCTION: @@ -206,6 +204,9 @@ unique_ptr Binder::Bind(TableRef &ref) { case TableReferenceType::PIVOT: result = Bind(ref.Cast()); break; + case TableReferenceType::SHOW_REF: + result = Bind(ref.Cast()); + break; case TableReferenceType::CTE: case TableReferenceType::INVALID: default: @@ -230,7 +231,7 @@ unique_ptr Binder::CreatePlan(BoundTableRef &ref) { case TableReferenceType::TABLE_FUNCTION: root = CreatePlan(ref.Cast()); break; - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: root = CreatePlan(ref.Cast()); break; case TableReferenceType::EXPRESSION_LIST: @@ -352,25 +353,25 @@ void Binder::AddCorrelatedColumn(const CorrelatedColumnInfo &info) { } } -bool Binder::HasMatchingBinding(const string &table_name, const string &column_name, string &error_message) { +bool Binder::HasMatchingBinding(const string &table_name, const string &column_name, ErrorData &error) { string empty_schema; - return HasMatchingBinding(empty_schema, table_name, column_name, error_message); + return HasMatchingBinding(empty_schema, table_name, column_name, error); } bool Binder::HasMatchingBinding(const string &schema_name, const string &table_name, const string &column_name, - string &error_message) { + ErrorData &error) { string empty_catalog; - return HasMatchingBinding(empty_catalog, schema_name, table_name, column_name, error_message); + return HasMatchingBinding(empty_catalog, schema_name, table_name, column_name, error); } bool Binder::HasMatchingBinding(const string &catalog_name, const string &schema_name, const string &table_name, - const string &column_name, string &error_message) { + const string &column_name, ErrorData &error) { optional_ptr binding; D_ASSERT(!lambda_bindings); if (macro_binding && table_name == macro_binding->alias) { binding = optional_ptr(macro_binding.get()); } else { - binding = bind_context.GetBinding(table_name, error_message); + binding = bind_context.GetBinding(table_name, error); } if (!binding) { @@ -394,7 +395,7 @@ bool Binder::HasMatchingBinding(const string &catalog_name, const string &schema bool binding_found; binding_found = binding->HasMatchingBinding(column_name); if (!binding_found) { - error_message = binding->ColumnNotFoundError(column_name); + error = binding->ColumnNotFoundError(column_name); } return binding_found; } @@ -414,6 +415,18 @@ void Binder::SetCanContainNulls(bool can_contain_nulls_p) { can_contain_nulls = can_contain_nulls_p; } +void Binder::SetAlwaysRequireRebind() { + reference current_binder = *this; + while (true) { + auto ¤t = current_binder.get(); + current.properties.always_require_rebind = true; + if (!current.parent) { + break; + } + current_binder = *current.parent; + } +} + void Binder::AddTableName(string table_name) { auto root_binder = GetRootBinder(); root_binder->table_names.insert(std::move(table_name)); @@ -424,19 +437,6 @@ const unordered_set &Binder::GetTableNames() { return root_binder->table_names; } -string Binder::FormatError(ParsedExpression &expr_context, const string &message) { - return FormatError(expr_context.query_location, message); -} - -string Binder::FormatError(TableRef &ref_context, const string &message) { - return FormatError(ref_context.query_location, message); -} - -string Binder::FormatErrorRecursive(idx_t query_location, const string &message, vector &values) { - QueryErrorContext context(root_statement, query_location); - return context.FormatErrorRecursive(message, values); -} - // FIXME: this is extremely naive void VerifyNotExcluded(ParsedExpression &expr) { if (expr.type == ExpressionType::COLUMN_REF) { @@ -489,7 +489,9 @@ BoundStatement Binder::BindReturning(vector> return result.types.push_back(result_type); projection_expressions.push_back(std::move(expr)); } - + if (new_returning_list.empty()) { + throw BinderException("RETURNING list is empty!"); + } auto projection = make_uniq(GenerateTableIndex(), std::move(projection_expressions)); projection->AddChild(std::move(child_operator)); D_ASSERT(result.types.size() == result.names.size()); diff --git a/src/duckdb/src/planner/binder/expression/bind_aggregate_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_aggregate_expression.cpp index ffb7d8b7a..2af12ca4e 100644 --- a/src/duckdb/src/planner/binder/expression/bind_aggregate_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_aggregate_expression.cpp @@ -86,7 +86,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu this->bound_aggregate = true; unique_ptr bound_filter; AggregateBinder aggregate_binder(binder, context); - string error; + ErrorData error; // Now we bind the filter (if any) if (aggr.filter) { @@ -114,7 +114,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu for (auto &child : aggr.children) { aggregate_binder.BindChild(child, 0, error); // We have to negate the fractions for PERCENTILE_XXXX DESC - if (error.empty() && ordered_set_agg) { + if (!error.HasError() && ordered_set_agg) { NegatePercentileFractions(context, child, negate_fractions); } } @@ -126,7 +126,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu } } - if (!error.empty()) { + if (error.HasError()) { // failed to bind child if (aggregate_binder.HasBoundColumns()) { for (idx_t i = 0; i < aggr.children.size(); i++) { @@ -136,7 +136,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu auto result = aggregate_binder.BindCorrelatedColumns(aggr.children[i], error); // if there is still an error after this, we could not successfully bind the aggregate if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } auto &bound_expr = BoundExpression::GetExpression(*aggr.children[i]); ExtractCorrelatedExpressions(binder, *bound_expr); @@ -145,7 +145,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu auto result = aggregate_binder.BindCorrelatedColumns(aggr.filter, error); // if there is still an error after this, we could not successfully bind the aggregate if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } auto &bound_expr = BoundExpression::GetExpression(*aggr.filter); ExtractCorrelatedExpressions(binder, *bound_expr); @@ -154,7 +154,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu for (auto &order : aggr.order_bys->orders) { auto result = aggregate_binder.BindCorrelatedColumns(order.expression, error); if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } auto &bound_expr = BoundExpression::GetExpression(*order.expression); ExtractCorrelatedExpressions(binder, *bound_expr); @@ -162,7 +162,7 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu } } else { // we didn't bind columns, try again in children - return BindResult(error); + return BindResult(std::move(error)); } } else if (depth > 0 && !aggregate_binder.HasBoundColumns()) { return BindResult("Aggregate with only constant parameters has to be bound in the root subquery"); @@ -207,7 +207,8 @@ BindResult BaseSelectBinder::BindAggregate(FunctionExpression &aggr, AggregateFu FunctionBinder function_binder(context); idx_t best_function = function_binder.BindFunction(func.name, func.functions, types, error); if (best_function == DConstants::INVALID_INDEX) { - throw BinderException(binder.FormatError(aggr, error)); + error.AddQueryLocation(aggr); + error.Throw(); } // found a matching function! auto bound_function = func.functions.GetFunctionByOffset(best_function); diff --git a/src/duckdb/src/planner/binder/expression/bind_between_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_between_expression.cpp index a44e5d378..b32dec197 100644 --- a/src/duckdb/src/planner/binder/expression/bind_between_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_between_expression.cpp @@ -5,31 +5,42 @@ #include "duckdb/planner/expression/bound_conjunction_expression.hpp" #include "duckdb/planner/expression/bound_parameter_expression.hpp" #include "duckdb/planner/expression_binder.hpp" +#include "duckdb/planner/binder.hpp" namespace duckdb { BindResult ExpressionBinder::BindExpression(BetweenExpression &expr, idx_t depth) { // first try to bind the children of the case expression - string error; + ErrorData error; BindChild(expr.input, depth, error); BindChild(expr.lower, depth, error); BindChild(expr.upper, depth, error); - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // the children have been successfully resolved auto &input = BoundExpression::GetExpression(*expr.input); auto &lower = BoundExpression::GetExpression(*expr.lower); auto &upper = BoundExpression::GetExpression(*expr.upper); - auto input_sql_type = input->return_type; - auto lower_sql_type = lower->return_type; - auto upper_sql_type = upper->return_type; + auto input_sql_type = ExpressionBinder::GetExpressionReturnType(*input); + auto lower_sql_type = ExpressionBinder::GetExpressionReturnType(*lower); + auto upper_sql_type = ExpressionBinder::GetExpressionReturnType(*upper); // cast the input types to the same type // now obtain the result type of the input types - auto input_type = BoundComparisonExpression::BindComparison(input_sql_type, lower_sql_type); - input_type = BoundComparisonExpression::BindComparison(input_type, upper_sql_type); + LogicalType input_type; + if (!BoundComparisonExpression::TryBindComparison(context, input_sql_type, lower_sql_type, input_type, expr.type)) { + + throw BinderException(expr, + "Cannot mix values of type %s and %s in BETWEEN clause - an explicit cast is required", + input_sql_type.ToString(), lower_sql_type.ToString()); + } + if (!BoundComparisonExpression::TryBindComparison(context, input_type, upper_sql_type, input_type, expr.type)) { + throw BinderException(expr, + "Cannot mix values of type %s and %s in BETWEEN clause - an explicit cast is required", + input_type.ToString(), upper_sql_type.ToString()); + } // add casts (if necessary) input = BoundCastExpression::AddCastToType(context, std::move(input), input_type); lower = BoundCastExpression::AddCastToType(context, std::move(lower), input_type); @@ -39,7 +50,7 @@ BindResult ExpressionBinder::BindExpression(BetweenExpression &expr, idx_t depth PushCollation(context, lower, input_type, false); PushCollation(context, upper, input_type, false); - if (!input->HasSideEffects() && !input->HasParameter() && !input->HasSubquery()) { + if (!input->IsVolatile() && !input->HasParameter() && !input->HasSubquery()) { // the expression does not have side effects and can be copied: create two comparisons // the reason we do this is that individual comparisons are easier to handle in optimizers // if both comparisons remain they will be folded together again into a single BETWEEN in the optimizer diff --git a/src/duckdb/src/planner/binder/expression/bind_case_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_case_expression.cpp index 6ba9bd8f3..a96a2e2bb 100644 --- a/src/duckdb/src/planner/binder/expression/bind_case_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_case_expression.cpp @@ -2,33 +2,38 @@ #include "duckdb/planner/expression/bound_case_expression.hpp" #include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression_binder.hpp" +#include "duckdb/planner/binder.hpp" namespace duckdb { BindResult ExpressionBinder::BindExpression(CaseExpression &expr, idx_t depth) { // first try to bind the children of the case expression - string error; + ErrorData error; for (auto &check : expr.case_checks) { BindChild(check.when_expr, depth, error); BindChild(check.then_expr, depth, error); } BindChild(expr.else_expr, depth, error); - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // the children have been successfully resolved // figure out the result type of the CASE expression auto &else_expr = BoundExpression::GetExpression(*expr.else_expr); - auto return_type = else_expr->return_type; + auto return_type = ExpressionBinder::GetExpressionReturnType(*else_expr); for (auto &check : expr.case_checks) { auto &then_expr = BoundExpression::GetExpression(*check.then_expr); - return_type = LogicalType::MaxLogicalType(return_type, then_expr->return_type); + auto then_type = ExpressionBinder::GetExpressionReturnType(*then_expr); + if (!LogicalType::TryGetMaxLogicalType(context, return_type, then_type, return_type)) { + throw BinderException( + expr, "Cannot mix values of type %s and %s in CASE expression - an explicit cast is required", + return_type.ToString(), then_type.ToString()); + } } // bind all the individual components of the CASE statement auto result = make_uniq(return_type); - for (idx_t i = 0; i < expr.case_checks.size(); i++) { - auto &check = expr.case_checks[i]; + for (auto &check : expr.case_checks) { auto &when_expr = BoundExpression::GetExpression(*check.when_expr); auto &then_expr = BoundExpression::GetExpression(*check.then_expr); BoundCaseCheck result_check; diff --git a/src/duckdb/src/planner/binder/expression/bind_cast_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_cast_expression.cpp index a62dd0084..68272d0a1 100644 --- a/src/duckdb/src/planner/binder/expression/bind_cast_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_cast_expression.cpp @@ -8,9 +8,9 @@ namespace duckdb { BindResult ExpressionBinder::BindExpression(CastExpression &expr, idx_t depth) { // first try to bind the child of the cast expression - string error = Bind(expr.child, depth); - if (!error.empty()) { - return BindResult(error); + auto error = Bind(expr.child, depth); + if (error.HasError()) { + return BindResult(std::move(error)); } // FIXME: We can also implement 'hello'::schema.custom_type; and pass by the schema down here. // Right now just considering its DEFAULT_SCHEMA always @@ -18,7 +18,7 @@ BindResult ExpressionBinder::BindExpression(CastExpression &expr, idx_t depth) { // the children have been successfully resolved auto &child = BoundExpression::GetExpression(*expr.child); if (expr.try_cast) { - if (child->return_type == expr.cast_type) { + if (ExpressionBinder::GetExpressionReturnType(*child) == expr.cast_type) { // no cast required: type matches return BindResult(std::move(child)); } diff --git a/src/duckdb/src/planner/binder/expression/bind_collate_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_collate_expression.cpp index 6682b5c94..7e0633a5a 100644 --- a/src/duckdb/src/planner/binder/expression/bind_collate_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_collate_expression.cpp @@ -6,9 +6,9 @@ namespace duckdb { BindResult ExpressionBinder::BindExpression(CollateExpression &expr, idx_t depth) { // first try to bind the child of the cast expression - string error = Bind(expr.child, depth); - if (!error.empty()) { - return BindResult(error); + auto error = Bind(expr.child, depth); + if (error.HasError()) { + return BindResult(std::move(error)); } auto &child = BoundExpression::GetExpression(*expr.child); if (child->HasParameter()) { 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 1f7113be4..20ec401d7 100644 --- a/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_columnref_expression.cpp @@ -54,7 +54,7 @@ unique_ptr ExpressionBinder::GetSQLValueFunction(const string return make_uniq(value_function, std::move(children)); } -unique_ptr ExpressionBinder::QualifyColumnName(const string &column_name, string &error_message) { +unique_ptr ExpressionBinder::QualifyColumnName(const string &column_name, ErrorData &error) { auto using_binding = binder.bind_context.GetUsingBinding(column_name); if (using_binding) { // we are referencing a USING column @@ -64,7 +64,7 @@ unique_ptr ExpressionBinder::QualifyColumnName(const string &c // we can! just assign the table name and re-bind return binder.bind_context.CreateColumnReference(using_binding->primary_binding, column_name); } else { - // // we cannot! we need to bind this as a coalesce between all the relevant columns + // we cannot! we need to bind this as COALESCE between all the relevant columns auto coalesce = make_uniq(ExpressionType::OPERATOR_COALESCE); coalesce->children.reserve(using_binding->bindings.size()); for (auto &entry : using_binding->bindings) { @@ -74,68 +74,80 @@ unique_ptr ExpressionBinder::QualifyColumnName(const string &c } } - // find a binding that contains this + // try binding as a lambda parameter + auto lambda_ref = LambdaRefExpression::FindMatchingBinding(lambda_bindings, column_name); + if (lambda_ref) { + return lambda_ref; + } + + // find a table binding that contains this column name string table_name = binder.bind_context.GetMatchingBinding(column_name); - // throw an error if a macro conflicts with a column name + // throw an error if a macro parameter name conflicts with a column name auto is_macro_column = false; - if (binder.macro_binding != nullptr && binder.macro_binding->HasMatchingBinding(column_name)) { + if (binder.macro_binding && binder.macro_binding->HasMatchingBinding(column_name)) { is_macro_column = true; if (!table_name.empty()) { throw BinderException("Conflicting column names for column " + column_name + "!"); } } - if (lambda_bindings) { - for (idx_t i = 0; i < lambda_bindings->size(); i++) { - if ((*lambda_bindings)[i].HasMatchingBinding(column_name)) { - - // throw an error if a lambda conflicts with a column name or a macro - if (!table_name.empty() || is_macro_column) { - throw BinderException("Conflicting column names for column " + column_name + "!"); - } - - D_ASSERT(!(*lambda_bindings)[i].alias.empty()); - return make_uniq(i, column_name); - } - } - } - + // bind as a macro column if (is_macro_column) { D_ASSERT(!binder.macro_binding->alias.empty()); return make_uniq(column_name, binder.macro_binding->alias); } - // see if it's a column - if (table_name.empty()) { - // column was not found - check if it is a SQL value function - auto value_function = GetSQLValueFunction(column_name); - if (value_function) { - return value_function; - } - // it's not, find candidates and error - auto similar_bindings = binder.bind_context.GetSimilarBindings(column_name); - string candidate_str = StringUtil::CandidatesMessage(similar_bindings, "Candidate bindings"); - error_message = - StringUtil::Format("Referenced column \"%s\" not found in FROM clause!%s", column_name, candidate_str); - return nullptr; + + // bind as a regular column + if (!table_name.empty()) { + return binder.bind_context.CreateColumnReference(table_name, column_name); } - return binder.bind_context.CreateColumnReference(table_name, column_name); + + // column was not found - check if it is a SQL value function + auto value_function = GetSQLValueFunction(column_name); + if (value_function) { + return value_function; + } + + // it's not, find candidates and error + auto similar_bindings = binder.bind_context.GetSimilarBindings(column_name); + error = ErrorData(BinderException::ColumnNotFound(column_name, similar_bindings)); + return nullptr; } -void ExpressionBinder::QualifyColumnNames(unique_ptr &expr) { +void ExpressionBinder::QualifyColumnNames(unique_ptr &expr, + vector> &lambda_params, + const bool within_function_expression) { + + bool next_within_function_expression = false; switch (expr->type) { case ExpressionType::COLUMN_REF: { - auto &colref = expr->Cast(); - string error_message; - auto new_expr = QualifyColumnName(colref, error_message); + + auto &col_ref = expr->Cast(); + + // don't qualify lambda parameters + if (LambdaExpression::IsLambdaParameter(lambda_params, col_ref.GetName())) { + return; + } + + ErrorData error; + auto new_expr = QualifyColumnName(col_ref, error); + if (new_expr) { if (!expr->alias.empty()) { + // Pre-existing aliases are added to the qualified column reference new_expr->alias = expr->alias; + } else if (within_function_expression) { + // Qualifying the column reference may add an alias, but this needs to be removed within function + // expressions, because the alias here means a named parameter instead of a positional parameter + new_expr->alias = ""; } - new_expr->query_location = colref.query_location; + + // replace the expression with the qualified column reference + new_expr->query_location = col_ref.query_location; expr = std::move(new_expr); } - break; + return; } case ExpressionType::POSITIONAL_REFERENCE: { auto &ref = expr->Cast(); @@ -148,60 +160,125 @@ void ExpressionBinder::QualifyColumnNames(unique_ptr &expr) { } break; } - default: + case ExpressionType::FUNCTION: { + // special-handling for lambdas, which are inside function expressions, + auto &function = expr->Cast(); + if (IsLambdaFunction(function)) { + // special case + return QualifyColumnNamesInLambda(function, lambda_params); + } + + next_within_function_expression = true; + break; + } + default: // fall through break; } - ParsedExpressionIterator::EnumerateChildren( - *expr, [&](unique_ptr &child) { QualifyColumnNames(child); }); + + // recurse on the child expressions + ParsedExpressionIterator::EnumerateChildren(*expr, [&](unique_ptr &child) { + QualifyColumnNames(child, lambda_params, next_within_function_expression); + }); +} + +void ExpressionBinder::QualifyColumnNamesInLambda(FunctionExpression &function, + vector> &lambda_params) { + + for (auto &child : function.children) { + if (child->expression_class != ExpressionClass::LAMBDA) { + // not a lambda expression + QualifyColumnNames(child, lambda_params, true); + continue; + } + + // special-handling for LHS lambda parameters + // we do not qualify them, and we add them to the lambda_params vector + auto &lambda_expr = child->Cast(); + string error_message; + auto column_ref_expressions = lambda_expr.ExtractColumnRefExpressions(error_message); + + if (!error_message.empty()) { + // possibly a JSON function, qualify both LHS and RHS + QualifyColumnNames(lambda_expr.lhs, lambda_params, true); + QualifyColumnNames(lambda_expr.expr, lambda_params, true); + continue; + } + + // push this level + lambda_params.emplace_back(); + + // push the lambda parameter names + for (const auto &column_ref_expr : column_ref_expressions) { + const auto &column_ref = column_ref_expr.get().Cast(); + lambda_params.back().emplace(column_ref.GetName()); + } + + // only qualify in RHS + QualifyColumnNames(lambda_expr.expr, lambda_params, true); + + // pop this level + lambda_params.pop_back(); + } } void ExpressionBinder::QualifyColumnNames(Binder &binder, unique_ptr &expr) { WhereBinder where_binder(binder, binder.context); - where_binder.QualifyColumnNames(expr); + vector> lambda_params; + where_binder.QualifyColumnNames(expr, lambda_params); } unique_ptr ExpressionBinder::CreateStructExtract(unique_ptr base, - string field_name) { + const string &field_name) { vector> children; children.push_back(std::move(base)); - children.push_back(make_uniq_base(Value(std::move(field_name)))); + children.push_back(make_uniq_base(Value(field_name))); auto extract_fun = make_uniq(ExpressionType::STRUCT_EXTRACT, std::move(children)); return std::move(extract_fun); } -unique_ptr ExpressionBinder::CreateStructPack(ColumnRefExpression &colref) { - D_ASSERT(colref.column_names.size() <= 3); - string error_message; - auto &table_name = colref.column_names.back(); - auto binding = binder.bind_context.GetBinding(table_name, error_message); +unique_ptr ExpressionBinder::CreateStructPack(ColumnRefExpression &col_ref) { + + D_ASSERT(col_ref.column_names.size() <= 3); + + // get a matching binding + ErrorData error; + auto &table_name = col_ref.column_names.back(); + auto binding = binder.bind_context.GetBinding(table_name, error); + if (!binding) { return nullptr; } - if (colref.column_names.size() >= 2) { + + if (col_ref.column_names.size() >= 2) { // "schema_name.table_name" auto catalog_entry = binding->GetStandardEntry(); if (!catalog_entry) { return nullptr; } + if (catalog_entry->name != table_name) { return nullptr; } - if (colref.column_names.size() == 2) { - auto &qualifier = colref.column_names[0]; + + if (col_ref.column_names.size() == 2) { + auto &qualifier = col_ref.column_names[0]; if (catalog_entry->catalog.GetName() != qualifier && catalog_entry->schema.name != qualifier) { return nullptr; } - } else if (colref.column_names.size() == 3) { - auto &catalog_name = colref.column_names[0]; - auto &schema_name = colref.column_names[1]; + + } else if (col_ref.column_names.size() == 3) { + auto &catalog_name = col_ref.column_names[0]; + auto &schema_name = col_ref.column_names[1]; if (catalog_entry->catalog.GetName() != catalog_name || catalog_entry->schema.name != schema_name) { return nullptr; } + } else { throw InternalException("Expected 2 or 3 column names for CreateStructPack"); } } + // We found the table, now create the struct_pack expression vector> child_expressions; child_expressions.reserve(binding->names.size()); @@ -211,122 +288,155 @@ unique_ptr ExpressionBinder::CreateStructPack(ColumnRefExpress return make_uniq("struct_pack", std::move(child_expressions)); } -unique_ptr ExpressionBinder::QualifyColumnName(ColumnRefExpression &colref, string &error_message) { - idx_t column_parts = colref.column_names.size(); +unique_ptr ExpressionBinder::QualifyColumnNameWithManyDots(ColumnRefExpression &col_ref, + ErrorData &error) { + + // two or more dots (i.e. "part1.part2.part3.part4...") + // -> part1 is a catalog, part2 is a schema, part3 is a table, part4 is a column name, part 5 and beyond are + // struct fields + // -> part1 is a catalog, part2 is a table, part3 is a column name, part4 and beyond are struct fields + // -> part1 is a schema, part2 is a table, part3 is a column name, part4 and beyond are struct fields + // -> part1 is a table, part2 is a column name, part3 and beyond are struct fields + // -> part1 is a column, part2 and beyond are struct fields + + // we always prefer the most top-level view + // i.e. in case of multiple resolution options, we resolve in order: + // -> 1. resolve "part1" as a catalog + // -> 2. resolve "part1" as a schema + // -> 3. resolve "part1" as a table + // -> 4. resolve "part1" as a column + + unique_ptr result_expr; + idx_t struct_extract_start; + + // first check if part1 is a catalog + if (col_ref.column_names.size() > 3 && + binder.HasMatchingBinding(col_ref.column_names[0], col_ref.column_names[1], col_ref.column_names[2], + col_ref.column_names[3], error)) { + // part1 is a catalog - the column reference is "catalog.schema.table.column" + result_expr = binder.bind_context.CreateColumnReference(col_ref.column_names[0], col_ref.column_names[1], + col_ref.column_names[2], col_ref.column_names[3]); + struct_extract_start = 4; + + } else if (binder.HasMatchingBinding(col_ref.column_names[0], INVALID_SCHEMA, col_ref.column_names[1], + col_ref.column_names[2], error)) { + // part1 is a catalog - the column reference is "catalog.table.column" + result_expr = binder.bind_context.CreateColumnReference(col_ref.column_names[0], INVALID_SCHEMA, + col_ref.column_names[1], col_ref.column_names[2]); + struct_extract_start = 3; + + } else if (binder.HasMatchingBinding(col_ref.column_names[0], col_ref.column_names[1], col_ref.column_names[2], + error)) { + // part1 is a schema - the column reference is "schema.table.column" + // any additional fields are turned into struct_extract calls + result_expr = binder.bind_context.CreateColumnReference(col_ref.column_names[0], col_ref.column_names[1], + col_ref.column_names[2]); + struct_extract_start = 3; + + } else if (binder.HasMatchingBinding(col_ref.column_names[0], col_ref.column_names[1], error)) { + // part1 is a table + // the column reference is "table.column" + // any additional fields are turned into struct_extract calls + result_expr = binder.bind_context.CreateColumnReference(col_ref.column_names[0], col_ref.column_names[1]); + struct_extract_start = 2; + + } else { + // part1 could be a column + ErrorData col_error; + result_expr = QualifyColumnName(col_ref.column_names[0], col_error); + if (!result_expr) { + // it is not! Try creating an implicit struct_pack + return CreateStructPack(col_ref); + } + // it is! add the struct extract calls + struct_extract_start = 1; + } + + // create a struct extract with all remaining column names + for (idx_t i = struct_extract_start; i < col_ref.column_names.size(); i++) { + result_expr = CreateStructExtract(std::move(result_expr), col_ref.column_names[i]); + } + + return result_expr; +} + +unique_ptr ExpressionBinder::QualifyColumnName(ColumnRefExpression &col_ref, ErrorData &error) { + + // try binding as a lambda parameter + if (!col_ref.IsQualified()) { + auto lambda_ref = LambdaRefExpression::FindMatchingBinding(lambda_bindings, col_ref.GetName()); + if (lambda_ref) { + return lambda_ref; + } + } + + idx_t column_parts = col_ref.column_names.size(); + // column names can have an arbitrary amount of dots // here is how the resolution works: if (column_parts == 1) { // no dots (i.e. "part1") // -> part1 refers to a column // check if we can qualify the column name with the table name - auto qualified_colref = QualifyColumnName(colref.GetColumnName(), error_message); - if (qualified_colref) { + auto qualified_col_ref = QualifyColumnName(col_ref.GetColumnName(), error); + if (qualified_col_ref) { // we could: return it - return qualified_colref; + return qualified_col_ref; } // we could not! Try creating an implicit struct_pack - return CreateStructPack(colref); - } else if (column_parts == 2) { + return CreateStructPack(col_ref); + } + + if (column_parts == 2) { // one dot (i.e. "part1.part2") // EITHER: // -> part1 is a table, part2 is a column // -> part1 is a column, part2 is a property of that column (i.e. struct_extract) - // first check if part1 is a table, and part2 is a standard column - if (binder.HasMatchingBinding(colref.column_names[0], colref.column_names[1], error_message)) { - // it is! return the colref directly - return binder.bind_context.CreateColumnReference(colref.column_names[0], colref.column_names[1]); - } else { - // otherwise check if we can turn this into a struct extract - string other_error; - auto qualified_colref = QualifyColumnName(colref.column_names[0], other_error); - if (qualified_colref) { - // we could: create a struct extract - return CreateStructExtract(std::move(qualified_colref), colref.column_names[1]); - } - // we could not! Try creating an implicit struct_pack - return CreateStructPack(colref); - } - } else { - // two or more dots (i.e. "part1.part2.part3.part4...") - // -> part1 is a catalog, part2 is a schema, part3 is a table, part4 is a column name, part 5 and beyond are - // struct fields - // -> part1 is a catalog, part2 is a table, part3 is a column name, part4 and beyond are struct fields - // -> part1 is a schema, part2 is a table, part3 is a column name, part4 and beyond are struct fields - // -> part1 is a table, part2 is a column name, part3 and beyond are struct fields - // -> part1 is a column, part2 and beyond are struct fields - - // we always prefer the most top-level view - // i.e. in case of multiple resolution options, we resolve in order: - // -> 1. resolve "part1" as a catalog - // -> 2. resolve "part1" as a schema - // -> 3. resolve "part1" as a table - // -> 4. resolve "part1" as a column - - unique_ptr result_expr; - idx_t struct_extract_start; - // first check if part1 is a catalog - if (colref.column_names.size() > 3 && - binder.HasMatchingBinding(colref.column_names[0], colref.column_names[1], colref.column_names[2], - colref.column_names[3], error_message)) { - // part1 is a catalog - the column reference is "catalog.schema.table.column" - result_expr = binder.bind_context.CreateColumnReference(colref.column_names[0], colref.column_names[1], - colref.column_names[2], colref.column_names[3]); - struct_extract_start = 4; - } else if (binder.HasMatchingBinding(colref.column_names[0], INVALID_SCHEMA, colref.column_names[1], - colref.column_names[2], error_message)) { - // part1 is a catalog - the column reference is "catalog.table.column" - result_expr = binder.bind_context.CreateColumnReference(colref.column_names[0], INVALID_SCHEMA, - colref.column_names[1], colref.column_names[2]); - struct_extract_start = 3; - } else if (binder.HasMatchingBinding(colref.column_names[0], colref.column_names[1], colref.column_names[2], - error_message)) { - // part1 is a schema - the column reference is "schema.table.column" - // any additional fields are turned into struct_extract calls - result_expr = binder.bind_context.CreateColumnReference(colref.column_names[0], colref.column_names[1], - colref.column_names[2]); - struct_extract_start = 3; - } else if (binder.HasMatchingBinding(colref.column_names[0], colref.column_names[1], error_message)) { - // part1 is a table - // the column reference is "table.column" - // any additional fields are turned into struct_extract calls - result_expr = binder.bind_context.CreateColumnReference(colref.column_names[0], colref.column_names[1]); - struct_extract_start = 2; - } else { - // part1 could be a column - string col_error; - result_expr = QualifyColumnName(colref.column_names[0], col_error); - if (!result_expr) { - // it is not! Try creating an implicit struct_pack - return CreateStructPack(colref); - } - // it is! add the struct extract calls - struct_extract_start = 1; + // first check if part1 is a table, and part2 is a standard column name + if (binder.HasMatchingBinding(col_ref.column_names[0], col_ref.column_names[1], error)) { + // it is! return the column reference directly + return binder.bind_context.CreateColumnReference(col_ref.column_names[0], col_ref.column_names[1]); } - for (idx_t i = struct_extract_start; i < colref.column_names.size(); i++) { - result_expr = CreateStructExtract(std::move(result_expr), colref.column_names[i]); + + // otherwise check if we can turn this into a struct extract + ErrorData other_error; + auto qualified_col_ref = QualifyColumnName(col_ref.column_names[0], other_error); + if (qualified_col_ref) { + // we could: create a struct extract + return CreateStructExtract(std::move(qualified_col_ref), col_ref.column_names[1]); } - return result_expr; + // we could not! Try creating an implicit struct_pack + return CreateStructPack(col_ref); } + + // three or more dots + return QualifyColumnNameWithManyDots(col_ref, error); } -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(LambdaRefExpression &lambda_ref, idx_t depth) { + D_ASSERT(lambda_bindings && lambda_ref.lambda_idx < lambda_bindings->size()); + return (*lambda_bindings)[lambda_ref.lambda_idx].Bind(lambda_ref, depth); } -BindResult ExpressionBinder::BindExpression(ColumnRefExpression &colref_p, idx_t depth) { +BindResult ExpressionBinder::BindExpression(ColumnRefExpression &col_ref_p, idx_t depth) { + if (binder.GetBindingMode() == BindingMode::EXTRACT_NAMES) { return BindResult(make_uniq(Value(LogicalType::SQLNULL))); } - string error_message; - auto expr = QualifyColumnName(colref_p, error_message); + + ErrorData error; + auto expr = QualifyColumnName(col_ref_p, error); if (!expr) { - return BindResult(binder.FormatError(colref_p, error_message)); + error.AddQueryLocation(col_ref_p); + return BindResult(std::move(error)); } - expr->query_location = colref_p.query_location; - // a generated column returns a generated expression, a struct on a column returns a struct extract + expr->query_location = col_ref_p.query_location; + + // the above QualifyColumName returns a generated expression for a generated + // column, and struct_extract for a struct, or a lambda reference expression, + // all of them are not column reference expressions, so we return here if (expr->type != ExpressionType::COLUMN_REF) { auto alias = expr->alias; auto result = BindExpression(expr, depth); @@ -336,37 +446,36 @@ BindResult ExpressionBinder::BindExpression(ColumnRefExpression &colref_p, idx_t return result; } - auto &colref = expr->Cast(); - D_ASSERT(colref.IsQualified()); - auto &table_name = colref.GetTableName(); - - // 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 - + // the above QualifyColumnName returned an individual column reference + // expression, which we resolve to either a base table or a subquery expression, + // and if it was a macro parameter, then we let macro_binding bind it to the argument BindResult result; + auto &col_ref = expr->Cast(); + D_ASSERT(col_ref.IsQualified()); + auto &table_name = col_ref.GetTableName(); if (binder.macro_binding && table_name == binder.macro_binding->alias) { - result = binder.macro_binding->Bind(colref, depth); + result = binder.macro_binding->Bind(col_ref, depth); } else { - result = binder.bind_context.BindColumn(colref, depth); + result = binder.bind_context.BindColumn(col_ref, depth); } - if (!result.HasError()) { - BoundColumnReferenceInfo ref; - ref.name = colref.column_names.back(); - ref.query_location = colref.query_location; - bound_columns.push_back(std::move(ref)); - } else { - result.error = binder.FormatError(colref_p, result.error); + if (result.HasError()) { + result.error.AddQueryLocation(col_ref_p); + return result; } + + // we bound the column reference + BoundColumnReferenceInfo ref; + ref.name = col_ref.column_names.back(); + ref.query_location = col_ref.query_location; + bound_columns.push_back(std::move(ref)); return result; } -bool ExpressionBinder::QualifyColumnAlias(const ColumnRefExpression &colref) { - // Only BaseSelectBinder will have a valid col alias map, - // otherwise just return false +bool ExpressionBinder::QualifyColumnAlias(const ColumnRefExpression &col_ref) { + // only the BaseSelectBinder will have a valid column alias map, + // otherwise we return false return false; } - } // namespace duckdb diff --git a/src/duckdb/src/planner/binder/expression/bind_comparison_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_comparison_expression.cpp index 0cff7da5b..522e85b6a 100644 --- a/src/duckdb/src/planner/binder/expression/bind_comparison_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_comparison_expression.cpp @@ -73,9 +73,63 @@ void ExpressionBinder::TestCollation(ClientContext &context, const string &colla PushCollation(context, expr, LogicalType::VARCHAR_COLLATION(collation)); } -LogicalType BoundComparisonExpression::BindComparison(LogicalType left_type, LogicalType right_type) { - auto result_type = LogicalType::MaxLogicalType(left_type, right_type); - switch (result_type.id()) { +static bool SwitchVarcharComparison(const LogicalType &type) { + switch (type.id()) { + case LogicalTypeId::TINYINT: + case LogicalTypeId::SMALLINT: + case LogicalTypeId::INTEGER: + case LogicalTypeId::BIGINT: + case LogicalTypeId::HUGEINT: + case LogicalTypeId::FLOAT: + case LogicalTypeId::DOUBLE: + case LogicalTypeId::DECIMAL: + case LogicalTypeId::UTINYINT: + case LogicalTypeId::USMALLINT: + case LogicalTypeId::UINTEGER: + case LogicalTypeId::UBIGINT: + case LogicalTypeId::UHUGEINT: + case LogicalTypeId::DATE: + case LogicalTypeId::TIME: + case LogicalTypeId::TIMESTAMP_SEC: + case LogicalTypeId::TIMESTAMP_MS: + case LogicalTypeId::TIMESTAMP: + case LogicalTypeId::TIMESTAMP_NS: + case LogicalTypeId::INTERVAL: + case LogicalTypeId::TIMESTAMP_TZ: + case LogicalTypeId::TIME_TZ: + case LogicalTypeId::INTEGER_LITERAL: + return true; + default: + return false; + } +} + +bool BoundComparisonExpression::TryBindComparison(ClientContext &context, const LogicalType &left_type, + const LogicalType &right_type, LogicalType &result_type, + ExpressionType comparison_type) { + LogicalType res; + bool is_equality; + switch (comparison_type) { + case ExpressionType::COMPARE_EQUAL: + case ExpressionType::COMPARE_NOTEQUAL: + case ExpressionType::COMPARE_IN: + case ExpressionType::COMPARE_NOT_IN: + case ExpressionType::COMPARE_DISTINCT_FROM: + case ExpressionType::COMPARE_NOT_DISTINCT_FROM: + is_equality = true; + break; + default: + is_equality = false; + break; + } + if (is_equality) { + res = LogicalType::ForceMaxLogicalType(left_type, right_type); + } else { + if (!LogicalType::TryGetMaxLogicalType(context, left_type, right_type, res)) { + return false; + } + } + switch (res.id()) { case LogicalTypeId::DECIMAL: { // result is a decimal: we need the maximum width and the maximum scale over width vector argument_types = {left_type, right_type}; @@ -84,7 +138,8 @@ LogicalType BoundComparisonExpression::BindComparison(LogicalType left_type, Log uint8_t width, scale; auto can_convert = argument_types[i].GetDecimalProperties(width, scale); if (!can_convert) { - return result_type; + result_type = res; + return true; } max_width = MaxValue(width, max_width); max_scale = MaxValue(scale, max_scale); @@ -95,14 +150,15 @@ LogicalType BoundComparisonExpression::BindComparison(LogicalType left_type, Log // target width does not fit in decimal: truncate the scale (if possible) to try and make it fit max_width = Decimal::MAX_WIDTH_DECIMAL; } - return LogicalType::DECIMAL(max_width, max_scale); + res = LogicalType::DECIMAL(max_width, max_scale); + break; } case LogicalTypeId::VARCHAR: // for comparison with strings, we prefer to bind to the numeric types - if (left_type.IsNumeric() || left_type.id() == LogicalTypeId::BOOLEAN) { - return left_type; - } else if (right_type.IsNumeric() || right_type.id() == LogicalTypeId::BOOLEAN) { - return right_type; + if (left_type.id() != LogicalTypeId::VARCHAR && SwitchVarcharComparison(left_type)) { + res = LogicalType::NormalizeType(left_type); + } else if (right_type.id() != LogicalTypeId::VARCHAR && SwitchVarcharComparison(right_type)) { + res = LogicalType::NormalizeType(right_type); } else { // else: check if collations are compatible auto left_collation = StringType::GetCollation(left_type); @@ -111,29 +167,59 @@ LogicalType BoundComparisonExpression::BindComparison(LogicalType left_type, Log throw BinderException("Cannot combine types with different collation!"); } } - return result_type; + break; default: - return result_type; + break; } + result_type = res; + return true; +} + +LogicalType BoundComparisonExpression::BindComparison(ClientContext &context, const LogicalType &left_type, + const LogicalType &right_type, ExpressionType comparison_type) { + LogicalType result_type; + if (!BoundComparisonExpression::TryBindComparison(context, left_type, right_type, result_type, comparison_type)) { + throw BinderException("Cannot mix values of type %s and %s - an explicit cast is required", + left_type.ToString(), right_type.ToString()); + } + return result_type; +} + +LogicalType ExpressionBinder::GetExpressionReturnType(const Expression &expr) { + if (expr.expression_class == ExpressionClass::BOUND_CONSTANT) { + if (expr.return_type == LogicalTypeId::VARCHAR && StringType::GetCollation(expr.return_type).empty()) { + return LogicalTypeId::STRING_LITERAL; + } + if (expr.return_type.IsIntegral()) { + auto &constant = expr.Cast(); + return LogicalType::INTEGER_LITERAL(constant.value); + } + } + return expr.return_type; } BindResult ExpressionBinder::BindExpression(ComparisonExpression &expr, idx_t depth) { // first try to bind the children of the case expression - string error; + ErrorData error; BindChild(expr.left, depth, error); BindChild(expr.right, depth, error); - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // the children have been successfully resolved auto &left = BoundExpression::GetExpression(*expr.left); auto &right = BoundExpression::GetExpression(*expr.right); - auto left_sql_type = left->return_type; - auto right_sql_type = right->return_type; + auto left_sql_type = ExpressionBinder::GetExpressionReturnType(*left); + auto right_sql_type = ExpressionBinder::GetExpressionReturnType(*right); // cast the input types to the same type // now obtain the result type of the input types - auto input_type = BoundComparisonExpression::BindComparison(left_sql_type, right_sql_type); + LogicalType input_type; + if (!BoundComparisonExpression::TryBindComparison(context, left_sql_type, right_sql_type, input_type, expr.type)) { + return BindResult(BinderException(expr, + "Cannot compare values of type %s and type %s - an explicit cast is required", + left_sql_type.ToString(), right_sql_type.ToString())); + } // add casts (if necessary) left = BoundCastExpression::AddCastToType(context, std::move(left), input_type, input_type.id() == LogicalTypeId::ENUM); diff --git a/src/duckdb/src/planner/binder/expression/bind_conjunction_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_conjunction_expression.cpp index 70b990a03..5125b1dba 100644 --- a/src/duckdb/src/planner/binder/expression/bind_conjunction_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_conjunction_expression.cpp @@ -7,12 +7,12 @@ namespace duckdb { BindResult ExpressionBinder::BindExpression(ConjunctionExpression &expr, idx_t depth) { // first try to bind the children of the case expression - string error; + ErrorData error; for (idx_t i = 0; i < expr.children.size(); i++) { BindChild(expr.children[i], depth, error); } - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // the children have been successfully resolved // cast the input types to boolean (if necessary) 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 66b1efc23..0d888e37b 100644 --- a/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_function_expression.cpp @@ -16,11 +16,28 @@ namespace duckdb { +BindResult ExpressionBinder::TryBindLambdaOrJson(FunctionExpression &function, idx_t depth, CatalogEntry &func) { + + 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.RawMessage() + + "\n" + " or: " + + json_bind_result.error.RawMessage()); +} + BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t depth, unique_ptr &expr_ptr) { // lookup the function in the catalog - QueryErrorContext error_context(binder.root_statement, function.query_location); - + QueryErrorContext error_context(function.query_location); binder.BindSchemaOrCatalog(function.catalog, function.schema); auto func = Catalog::GetEntry(context, CatalogType::SCALAR_FUNCTION_ENTRY, function.catalog, function.schema, function.function_name, OnEntryNotFound::RETURN_NULL, error_context); @@ -30,16 +47,15 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t Catalog::GetEntry(context, CatalogType::TABLE_FUNCTION_ENTRY, function.catalog, function.schema, function.function_name, OnEntryNotFound::RETURN_NULL, error_context); if (table_func) { - throw BinderException(binder.FormatError( - function, - StringUtil::Format("Function \"%s\" is a table function but it was used as a scalar function. This " - "function has to be called in a FROM clause (similar to a table).", - function.function_name))); + throw BinderException(function, + "Function \"%s\" is a table function but it was used as a scalar function. This " + "function has to be called in a FROM clause (similar to a table).", + function.function_name); } // not a table function - check if the schema is set if (!function.schema.empty()) { // the schema is set - check if we can turn this the schema into a column ref - string error; + ErrorData error; unique_ptr colref; if (function.catalog.empty()) { colref = make_uniq(function.schema); @@ -47,7 +63,7 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t colref = make_uniq(function.schema, function.catalog); } auto new_colref = QualifyColumnName(*colref, error); - bool is_col = error.empty() ? true : false; + bool is_col = !error.HasError(); bool is_col_alias = QualifyColumnAlias(*colref); if (is_col || is_col_alias) { @@ -73,36 +89,11 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t switch (func->type) { case CatalogType::SCALAR_FUNCTION_ENTRY: { // scalar function - - // check for lambda parameters, ignore ->> operator (JSON extension) - bool try_bind_lambda = false; - if (function.function_name != "->>") { - for (auto &child : function.children) { - if (child->expression_class == ExpressionClass::LAMBDA) { - try_bind_lambda = true; - } - } - } - - // other scalar function - if (!try_bind_lambda) { - return BindFunction(function, func->Cast(), depth); - } - - auto lambda_bind_result = BindLambdaFunction(function, func->Cast(), depth); - if (!lambda_bind_result.HasError()) { - return lambda_bind_result; + if (IsLambdaFunction(function)) { + // special case + return TryBindLambdaOrJson(function, depth, *func); } - - 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); + return BindFunction(function, func->Cast(), depth); } case CatalogType::MACRO_ENTRY: // macro function @@ -114,17 +105,16 @@ BindResult ExpressionBinder::BindExpression(FunctionExpression &function, idx_t } BindResult ExpressionBinder::BindFunction(FunctionExpression &function, ScalarFunctionCatalogEntry &func, idx_t depth) { - // bind the children of the function expression - string error; + ErrorData error; // bind of each child for (idx_t i = 0; i < function.children.size(); i++) { BindChild(function.children[i], depth, error); } - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } if (binder.GetBindingMode() == BindingMode::EXTRACT_NAMES) { return BindResult(make_uniq(Value(LogicalType::SQLNULL))); @@ -139,10 +129,16 @@ BindResult ExpressionBinder::BindFunction(FunctionExpression &function, ScalarFu } FunctionBinder function_binder(context); - unique_ptr result = - function_binder.BindScalarFunction(func, std::move(children), error, function.is_operator, &binder); + auto result = function_binder.BindScalarFunction(func, std::move(children), error, function.is_operator, &binder); if (!result) { - throw BinderException(binder.FormatError(function, error)); + error.AddQueryLocation(function); + error.Throw(); + } + if (result->type == ExpressionType::BOUND_FUNCTION) { + auto &bound_function = result->Cast(); + if (bound_function.function.stability == FunctionStability::CONSISTENT_WITHIN_QUERY) { + binder.SetAlwaysRequireRebind(); + } } return BindResult(std::move(result)); } @@ -168,10 +164,10 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc D_ASSERT(function.children[1]->GetExpressionClass() == ExpressionClass::LAMBDA); // bind the list parameter - string error; + ErrorData error; BindChild(function.children[0], depth, error); - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // get the logical type of the children of the list @@ -229,7 +225,8 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc unique_ptr result = function_binder.BindScalarFunction(func, std::move(children), error, function.is_operator, &binder); if (!result) { - throw BinderException(binder.FormatError(function, error)); + error.AddQueryLocation(function); + error.Throw(); } auto &bound_function_expr = result->Cast(); @@ -269,11 +266,11 @@ BindResult ExpressionBinder::BindLambdaFunction(FunctionExpression &function, Sc BindResult ExpressionBinder::BindAggregate(FunctionExpression &expr, AggregateFunctionCatalogEntry &function, idx_t depth) { - return BindResult(binder.FormatError(expr, UnsupportedAggregateMessage())); + return BindResult(BinderException(expr, UnsupportedAggregateMessage())); } BindResult ExpressionBinder::BindUnnest(FunctionExpression &expr, idx_t depth, bool root_expression) { - return BindResult(binder.FormatError(expr, UnsupportedUnnestMessage())); + return BindResult(BinderException(expr, UnsupportedUnnestMessage())); } string ExpressionBinder::UnsupportedAggregateMessage() { diff --git a/src/duckdb/src/planner/binder/expression/bind_lambda.cpp b/src/duckdb/src/planner/binder/expression/bind_lambda.cpp index 1d7a33b54..2907889a9 100644 --- a/src/duckdb/src/planner/binder/expression/bind_lambda.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_lambda.cpp @@ -35,6 +35,33 @@ idx_t GetLambdaParamIndex(const vector &lambda_bindings, const Bou return offset; } +void ExtractParameter(ParsedExpression &expr, vector &column_names, vector &column_aliases) { + + auto &column_ref = expr.Cast(); + if (column_ref.IsQualified()) { + throw BinderException(LambdaExpression::InvalidParametersErrorMessage()); + } + + column_names.push_back(column_ref.GetName()); + column_aliases.push_back(column_ref.ToString()); +} + +void ExtractParameters(LambdaExpression &expr, vector &column_names, vector &column_aliases) { + + // extract the lambda parameters, which are a single column + // reference, or a list of column references (ROW function) + string error_message; + auto column_refs = expr.ExtractColumnRefExpressions(error_message); + if (!error_message.empty()) { + throw BinderException(error_message); + } + + for (const auto &column_ref : column_refs) { + ExtractParameter(column_ref.get(), column_names, column_aliases); + } + D_ASSERT(!column_names.empty()); +} + BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, const LogicalType &list_child_type, optional_ptr bind_lambda_function) { @@ -45,70 +72,19 @@ BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, return BindExpression(arrow_expr, depth); } - // binding the lambda expression - D_ASSERT(expr.lhs); - if (expr.lhs->expression_class != ExpressionClass::FUNCTION && - expr.lhs->expression_class != ExpressionClass::COLUMN_REF) { - throw BinderException( - "Invalid parameter list! Parameters must be comma-separated column names, e.g. x or (x, y)."); - } - - // move the lambda parameters to the params vector - if (expr.lhs->expression_class == ExpressionClass::COLUMN_REF) { - expr.params.push_back(std::move(expr.lhs)); - } else { - auto &func_expr = expr.lhs->Cast(); - for (idx_t i = 0; i < func_expr.children.size(); i++) { - expr.params.push_back(std::move(func_expr.children[i])); - } - } - D_ASSERT(!expr.params.empty()); - - // create dummy columns for the lambda parameters (lhs) + // extract and verify lambda parameters to create dummy columns vector column_types; vector column_names; - vector params_strings; - - // 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("Lambda parameter must be a column name."); - } - - auto column_ref = expr.params[i]->Cast(); - if (column_ref.IsQualified()) { - throw BinderException("Invalid lambda parameter name '%s': must be unqualified", column_ref.ToString()); - } - + vector column_aliases; + ExtractParameters(expr, column_names, column_aliases); + for (idx_t i = 0; i < column_names.size(); i++) { 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()); - } - - // ensure that we do not have ambiguous lambda parameters - if (lambda_bindings) { - for (const auto &binding : *lambda_bindings) { - for (const auto &outer_lambda_parameter : binding.names) { - for (const auto &this_lambda_parameter : column_names) { - if (outer_lambda_parameter == this_lambda_parameter) { - throw BinderException("Ambiguous lambda parameter name: '%s'. Try changing your lambda " - "parameter name. \n Some list functions use lambda functions " - "under the hood, so *the same* function cannot be nested, like " - "list_intersect(list_intersect(...),...), list_has_any, list_has_all, " - "and their aliases. \n " - "Try writing them out manually with lambda functions to define explicit " - "lambda parameter names.", - outer_lambda_parameter); - } - } - } - } } // base table alias - auto params_alias = StringUtil::Join(params_strings, ", "); - if (params_strings.size() > 1) { - params_alias = "(" + params_alias + ")"; + auto table_alias = StringUtil::Join(column_aliases, ", "); + if (column_aliases.size() > 1) { + table_alias = "(" + table_alias + ")"; } // create a lambda binding and push it to the lambda bindings vector @@ -116,17 +92,9 @@ BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, if (!lambda_bindings) { lambda_bindings = &local_bindings; } - DummyBinding new_lambda_binding(column_types, column_names, params_alias); + DummyBinding new_lambda_binding(column_types, column_names, table_alias); lambda_bindings->push_back(new_lambda_binding); - // bind the parameter expressions - for (idx_t i = 0; i < expr.params.size(); i++) { - auto result = BindExpression(expr.params[i], depth, false); - if (result.HasError()) { - throw InternalException("Error during lambda binding: %s", result.error); - } - } - auto result = BindExpression(expr.expr, depth, false); lambda_bindings->pop_back(); @@ -137,11 +105,11 @@ BindResult ExpressionBinder::BindExpression(LambdaExpression &expr, idx_t depth, } if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } return BindResult(make_uniq(ExpressionType::LAMBDA, LogicalType::LAMBDA, - std::move(result.expression), params_strings.size())); + std::move(result.expression), column_names.size())); } void ExpressionBinder::TransformCapturedLambdaColumn(unique_ptr &original, diff --git a/src/duckdb/src/planner/binder/expression/bind_macro_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_macro_expression.cpp index cce36d49d..ced09d180 100644 --- a/src/duckdb/src/planner/binder/expression/bind_macro_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_macro_expression.cpp @@ -9,43 +9,107 @@ namespace duckdb { -void ExpressionBinder::ReplaceMacroParametersRecursive(unique_ptr &expr) { +void ExpressionBinder::ReplaceMacroParametersInLambda(FunctionExpression &function, + vector> &lambda_params) { + + for (auto &child : function.children) { + if (child->expression_class != ExpressionClass::LAMBDA) { + // not a lambda expression + ReplaceMacroParameters(child, lambda_params); + continue; + } + + // special-handling for LHS lambda parameters + // we do not replace them, and we add them to the lambda_params vector + auto &lambda_expr = child->Cast(); + string error_message; + auto column_ref_expressions = lambda_expr.ExtractColumnRefExpressions(error_message); + + if (!error_message.empty()) { + // possibly a JSON function, replace both LHS and RHS + ParsedExpressionIterator::EnumerateChildren(*lambda_expr.lhs, [&](unique_ptr &child) { + ReplaceMacroParameters(child, lambda_params); + }); + ParsedExpressionIterator::EnumerateChildren(*lambda_expr.expr, [&](unique_ptr &child) { + ReplaceMacroParameters(child, lambda_params); + }); + continue; + } + + // push this level + lambda_params.emplace_back(); + + // push the lambda parameter names + for (const auto &column_ref_expr : column_ref_expressions) { + const auto &column_ref = column_ref_expr.get().Cast(); + lambda_params.back().emplace(column_ref.GetName()); + } + + // only replace in RHS + ParsedExpressionIterator::EnumerateChildren(*lambda_expr.expr, [&](unique_ptr &child) { + ReplaceMacroParameters(child, lambda_params); + }); + + // pop this level + lambda_params.pop_back(); + } +} + +void ExpressionBinder::ReplaceMacroParameters(unique_ptr &expr, + vector> &lambda_params) { + switch (expr->GetExpressionClass()) { case ExpressionClass::COLUMN_REF: { - // if expr is a parameter, replace it with its argument - auto &colref = expr->Cast(); + // if the expression is a parameter, replace it with its argument + auto &col_ref = expr->Cast(); + + // don't replace lambda parameters + if (LambdaExpression::IsLambdaParameter(lambda_params, col_ref.GetName())) { + return; + } + bool bind_macro_parameter = false; - if (colref.IsQualified()) { - bind_macro_parameter = false; - if (colref.GetTableName().find(DummyBinding::DUMMY_NAME) != string::npos) { + if (col_ref.IsQualified()) { + if (col_ref.GetTableName().find(DummyBinding::DUMMY_NAME) != string::npos) { bind_macro_parameter = true; } } else { - bind_macro_parameter = macro_binding->HasMatchingBinding(colref.GetColumnName()); + bind_macro_parameter = macro_binding->HasMatchingBinding(col_ref.GetColumnName()); } + if (bind_macro_parameter) { - D_ASSERT(macro_binding->HasMatchingBinding(colref.GetColumnName())); - expr = macro_binding->ParamToArg(colref); + D_ASSERT(macro_binding->HasMatchingBinding(col_ref.GetColumnName())); + expr = macro_binding->ParamToArg(col_ref); } return; } + case ExpressionClass::FUNCTION: { + // special-handling for lambdas, which are inside function expressions, + auto &function = expr->Cast(); + if (IsLambdaFunction(function)) { + // special case + return ReplaceMacroParametersInLambda(function, lambda_params); + } + break; + } case ExpressionClass::SUBQUERY: { - // replacing parameters within a subquery is slightly different auto &sq = (expr->Cast()).subquery; ParsedExpressionIterator::EnumerateQueryNodeChildren( - *sq->node, [&](unique_ptr &child) { ReplaceMacroParametersRecursive(child); }); + *sq->node, [&](unique_ptr &child) { ReplaceMacroParameters(child, lambda_params); }); break; } default: // fall through break; } - // unfold child expressions + + // replace macro parameters in child expressions ParsedExpressionIterator::EnumerateChildren( - *expr, [&](unique_ptr &child) { ReplaceMacroParametersRecursive(child); }); + *expr, [&](unique_ptr &child) { ReplaceMacroParameters(child, lambda_params); }); } BindResult ExpressionBinder::BindMacro(FunctionExpression &function, ScalarMacroCatalogEntry ¯o_func, idx_t depth, unique_ptr &expr) { + // recast function so we can access the scalar member function->expression auto ¯o_def = macro_func.function->Cast(); @@ -56,7 +120,7 @@ BindResult ExpressionBinder::BindMacro(FunctionExpression &function, ScalarMacro string error = MacroFunction::ValidateArguments(*macro_func.function, macro_func.name, function, positionals, defaults); if (!error.empty()) { - throw BinderException(binder.FormatError(*expr, error)); + throw BinderException(*expr, error); } // create a MacroBinding to bind this macro's parameters to its arguments @@ -82,8 +146,14 @@ BindResult ExpressionBinder::BindMacro(FunctionExpression &function, ScalarMacro // replace current expression with stored macro expression expr = macro_def.expression->Copy(); + // qualify only the macro parameters with a new empty binder that only knows the macro binding + auto dummy_binder = Binder::CreateBinder(context); + dummy_binder->macro_binding = new_macro_binding.get(); + ExpressionBinder::QualifyColumnNames(*dummy_binder, expr); + // now replace the parameters - ReplaceMacroParametersRecursive(expr); + vector> lambda_params; + ReplaceMacroParameters(expr, lambda_params); // bind the unfolded macro return BindExpression(expr, depth); diff --git a/src/duckdb/src/planner/binder/expression/bind_operator_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_operator_expression.cpp index 0b45d7364..4aaac887f 100644 --- a/src/duckdb/src/planner/binder/expression/bind_operator_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_operator_expression.cpp @@ -1,69 +1,63 @@ +#include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/parser/expression/operator_expression.hpp" +#include "duckdb/planner/expression/bound_case_expression.hpp" #include "duckdb/planner/expression/bound_cast_expression.hpp" +#include "duckdb/planner/expression/bound_comparison_expression.hpp" +#include "duckdb/planner/binder.hpp" +#include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/planner/expression/bound_operator_expression.hpp" -#include "duckdb/planner/expression/bound_case_expression.hpp" #include "duckdb/planner/expression/bound_parameter_expression.hpp" -#include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/planner/expression_binder.hpp" -#include "duckdb/planner/expression/bound_comparison_expression.hpp" namespace duckdb { -static LogicalType ResolveNotType(OperatorExpression &op, vector> &children) { +LogicalType ExpressionBinder::ResolveNotType(OperatorExpression &op, vector> &children) { // NOT expression, cast child to BOOLEAN D_ASSERT(children.size() == 1); - children[0] = BoundCastExpression::AddDefaultCastToType(std::move(children[0]), LogicalType::BOOLEAN); + children[0] = BoundCastExpression::AddCastToType(context, std::move(children[0]), LogicalType::BOOLEAN); return LogicalType(LogicalTypeId::BOOLEAN); } -static LogicalType ResolveInType(OperatorExpression &op, vector> &children, - ClientContext &context) { +LogicalType ExpressionBinder::ResolveInType(OperatorExpression &op, vector> &children) { if (children.empty()) { throw InternalException("IN requires at least a single child node"); } // get the maximum type from the children - LogicalType max_type = children[0]->return_type; - bool any_varchar = children[0]->return_type == LogicalType::VARCHAR; - bool any_enum = children[0]->return_type.id() == LogicalTypeId::ENUM; + LogicalType max_type = ExpressionBinder::GetExpressionReturnType(*children[0]); bool is_in_operator = (op.type == ExpressionType::COMPARE_IN || op.type == ExpressionType::COMPARE_NOT_IN); for (idx_t i = 1; i < children.size(); i++) { + auto child_return = ExpressionBinder::GetExpressionReturnType(*children[i]); if (is_in_operator) { // If it's IN/NOT_IN operator, adjust DECIMAL and VARCHAR returned type. - max_type = BoundComparisonExpression::BindComparison(max_type, children[i]->return_type); + if (!BoundComparisonExpression::TryBindComparison(context, max_type, child_return, max_type, op.type)) { + throw BinderException(op, + "Cannot mix values of type %s and %s in %s clause - an explicit cast is required", + max_type.ToString(), child_return.ToString(), + op.type == ExpressionType::COMPARE_IN ? "IN" : "NOT IN"); + } } else { // If it's COALESCE operator, don't do extra adjustment. - max_type = LogicalType::MaxLogicalType(max_type, children[i]->return_type); - } - if (children[i]->return_type == LogicalType::VARCHAR) { - any_varchar = true; + if (!LogicalType::TryGetMaxLogicalType(context, max_type, child_return, max_type)) { + throw BinderException( + op, "Cannot mix values of type %s and %s in COALESCE operator - an explicit cast is required", + max_type.ToString(), child_return.ToString()); + } } - if (children[i]->return_type.id() == LogicalTypeId::ENUM) { - any_enum = true; - } - } - - // If max_type is already VARCHAR, no need to adjust it incase it's IN/NOT_IN operator. Reassignment of max_type - // will cause collation information lost. - if (any_varchar && any_enum && max_type.id() != LogicalTypeId::VARCHAR) { - // For the coalesce function, we must be sure we always upcast the parameters to VARCHAR, if there are at least - // one enum and one varchar - max_type = LogicalType::VARCHAR; } // cast all children to the same type - for (idx_t i = 0; i < children.size(); i++) { - children[i] = BoundCastExpression::AddDefaultCastToType(std::move(children[i]), max_type); + for (auto &child : children) { + child = BoundCastExpression::AddCastToType(context, std::move(child), max_type); if (is_in_operator) { // If it's IN/NOT_IN operator, push collation functions. - ExpressionBinder::PushCollation(context, children[i], max_type, true); + ExpressionBinder::PushCollation(context, child, max_type, true); } } // (NOT) IN always returns a boolean return LogicalType::BOOLEAN; } -static LogicalType ResolveOperatorType(OperatorExpression &op, vector> &children, - ClientContext &context) { +LogicalType ExpressionBinder::ResolveOperatorType(OperatorExpression &op, vector> &children) { switch (op.type) { case ExpressionType::OPERATOR_IS_NULL: case ExpressionType::OPERATOR_IS_NOT_NULL: @@ -74,9 +68,9 @@ static LogicalType ResolveOperatorType(OperatorExpression &op, vectorreturn_type; } case ExpressionType::OPERATOR_NOT: @@ -95,12 +89,12 @@ BindResult ExpressionBinder::BindExpression(OperatorExpression &op, idx_t depth) return BindGroupingFunction(op, depth); } // bind the children of the operator expression - string error; + ErrorData error; for (idx_t i = 0; i < op.children.size(); i++) { BindChild(op.children[i], depth, error); } - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } // all children bound successfully string function_name; @@ -108,8 +102,20 @@ BindResult ExpressionBinder::BindExpression(OperatorExpression &op, idx_t depth) case ExpressionType::ARRAY_EXTRACT: { D_ASSERT(op.children[0]->expression_class == ExpressionClass::BOUND_EXPRESSION); auto &b_exp = BoundExpression::GetExpression(*op.children[0]); - if (b_exp->return_type.id() == LogicalTypeId::MAP) { + const auto &b_exp_type = b_exp->return_type; + if (b_exp_type.id() == LogicalTypeId::MAP) { function_name = "map_extract"; + } else if (b_exp_type.IsJSONType() && op.children.size() == 2) { + function_name = "json_extract"; + // Make sure we only extract array elements, not fields, by adding the $[] syntax + auto &i_exp = BoundExpression::GetExpression(*op.children[1]); + if (i_exp->GetExpressionClass() == ExpressionClass::BOUND_CONSTANT) { + auto &const_exp = i_exp->Cast(); + if (!const_exp.value.IsNull()) { + const_exp.value = StringUtil::Format("$[%s]", const_exp.value.ToString()); + const_exp.return_type = LogicalType::VARCHAR; + } + } } else { function_name = "array_extract"; } @@ -124,14 +130,28 @@ BindResult ExpressionBinder::BindExpression(OperatorExpression &op, idx_t depth) D_ASSERT(op.children[1]->expression_class == ExpressionClass::BOUND_EXPRESSION); auto &extract_exp = BoundExpression::GetExpression(*op.children[0]); auto &name_exp = BoundExpression::GetExpression(*op.children[1]); - auto extract_expr_type = extract_exp->return_type.id(); - if (extract_expr_type != LogicalTypeId::STRUCT && extract_expr_type != LogicalTypeId::UNION && - extract_expr_type != LogicalTypeId::SQLNULL) { + const auto &extract_expr_type = extract_exp->return_type; + if (extract_expr_type.id() != LogicalTypeId::STRUCT && extract_expr_type.id() != LogicalTypeId::UNION && + extract_expr_type.id() != LogicalTypeId::SQLNULL && !extract_expr_type.IsJSONType()) { return BindResult(StringUtil::Format( - "Cannot extract field %s from expression \"%s\" because it is not a struct or a union", + "Cannot extract field %s from expression \"%s\" because it is not a struct, union, or json", name_exp->ToString(), extract_exp->ToString())); } - function_name = extract_expr_type == LogicalTypeId::UNION ? "union_extract" : "struct_extract"; + if (extract_expr_type.id() == LogicalTypeId::UNION) { + function_name = "union_extract"; + } else if (extract_expr_type.IsJSONType()) { + function_name = "json_extract"; + // Make sure we only extract fields, not array elements, by adding $. syntax + if (name_exp->GetExpressionClass() == ExpressionClass::BOUND_CONSTANT) { + auto &const_exp = name_exp->Cast(); + if (!const_exp.value.IsNull()) { + const_exp.value = StringUtil::Format("$.\"%s\"", const_exp.value.ToString()); + const_exp.return_type = LogicalType::VARCHAR; + } + } + } else { + function_name = "struct_extract"; + } break; } case ExpressionType::ARRAY_CONSTRUCTOR: @@ -154,7 +174,7 @@ BindResult ExpressionBinder::BindExpression(OperatorExpression &op, idx_t depth) children.push_back(std::move(BoundExpression::GetExpression(*op.children[i]))); } // now resolve the types - LogicalType result_type = ResolveOperatorType(op, children, context); + LogicalType result_type = ResolveOperatorType(op, children); if (op.type == ExpressionType::OPERATOR_COALESCE) { if (children.empty()) { throw BinderException("COALESCE needs at least one child"); diff --git a/src/duckdb/src/planner/binder/expression/bind_star_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_star_expression.cpp index c0a25d937..915c32e22 100644 --- a/src/duckdb/src/planner/binder/expression/bind_star_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_star_expression.cpp @@ -56,8 +56,8 @@ bool Binder::FindStarExpression(unique_ptr &expr, StarExpressi if (*star) { // we can have multiple if (!(*star)->Equals(current_star)) { - throw BinderException( - FormatError(*expr, "Multiple different STAR/COLUMNS in the same expression are not supported")); + throw BinderException(*expr, + "Multiple different STAR/COLUMNS in the same expression are not supported"); } return true; } @@ -123,7 +123,7 @@ void Binder::ExpandStarExpression(unique_ptr expr, duckdb_re2::RE2 regex(regex_str); if (!regex.error().empty()) { auto err = StringUtil::Format("Failed to compile regex \"%s\": %s", regex_str, regex.error()); - throw BinderException(FormatError(*star, err)); + throw BinderException(*star, err); } vector> new_list; for (idx_t i = 0; i < star_list.size(); i++) { @@ -135,7 +135,7 @@ void Binder::ExpandStarExpression(unique_ptr expr, } if (new_list.empty()) { auto err = StringUtil::Format("No matching columns found that match regex \"%s\"", regex_str); - throw BinderException(FormatError(*star, err)); + throw BinderException(*star, err); } star_list = std::move(new_list); } else if (val.type().id() == LogicalTypeId::LIST && @@ -144,7 +144,7 @@ void Binder::ExpandStarExpression(unique_ptr expr, if (val.IsNull() || ListValue::GetChildren(val).empty()) { auto err = StringUtil::Format("Star expression \"%s\" resulted in an empty set of columns", star->ToString()); - throw BinderException(FormatError(*star, err)); + throw BinderException(*star, err); } auto &children = ListValue::GetChildren(val); vector> new_list; @@ -171,8 +171,8 @@ void Binder::ExpandStarExpression(unique_ptr expr, } star_list = std::move(new_list); } else { - throw BinderException(FormatError( - *star, "COLUMNS expects either a VARCHAR argument (regex) or a LIST of VARCHAR (list of columns)")); + throw BinderException( + *star, "COLUMNS expects either a VARCHAR argument (regex) or a LIST of VARCHAR (list of columns)"); } } diff --git a/src/duckdb/src/planner/binder/expression/bind_subquery_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_subquery_expression.cpp index 9aa09ea5c..dc0f7624c 100644 --- a/src/duckdb/src/planner/binder/expression/bind_subquery_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_subquery_expression.cpp @@ -55,8 +55,7 @@ BindResult ExpressionBinder::BindExpression(SubqueryExpression &expr, idx_t dept } } if (expr.subquery_type != SubqueryType::EXISTS && bound_node->types.size() > 1) { - throw BinderException(binder.FormatError( - expr, StringUtil::Format("Subquery returns %zu columns - expected 1", bound_node->types.size()))); + throw BinderException(expr, "Subquery returns %zu columns - expected 1", bound_node->types.size()); } auto prior_subquery = std::move(expr.subquery); expr.subquery = make_uniq(); @@ -66,9 +65,9 @@ BindResult ExpressionBinder::BindExpression(SubqueryExpression &expr, idx_t dept // now bind the child node of the subquery if (expr.child) { // first bind the children of the subquery, if any - string error = Bind(expr.child, depth); - if (!error.empty()) { - return BindResult(error); + auto error = Bind(expr.child, depth); + if (error.HasError()) { + return BindResult(std::move(error)); } } // both binding the child and binding the subquery was successful @@ -88,7 +87,13 @@ BindResult ExpressionBinder::BindExpression(SubqueryExpression &expr, idx_t dept // cast child and subquery child to equivalent types D_ASSERT(bound_node->types.size() == 1); auto &child = BoundExpression::GetExpression(*expr.child); - auto compare_type = LogicalType::MaxLogicalType(child->return_type, bound_node->types[0]); + auto child_type = ExpressionBinder::GetExpressionReturnType(*child); + LogicalType compare_type; + if (!LogicalType::TryGetMaxLogicalType(context, child_type, bound_node->types[0], compare_type)) { + throw BinderException( + expr, "Cannot compare values of type %s and %s in IN/ANY/ALL clause - an explicit cast is required", + child_type.ToString(), bound_node->types[0]); + } child = BoundCastExpression::AddCastToType(context, std::move(child), compare_type); result->child_type = bound_node->types[0]; result->child_target = compare_type; diff --git a/src/duckdb/src/planner/binder/expression/bind_unnest_expression.cpp b/src/duckdb/src/planner/binder/expression/bind_unnest_expression.cpp index 6ae23c95a..59ddc83b5 100644 --- a/src/duckdb/src/planner/binder/expression/bind_unnest_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_unnest_expression.cpp @@ -32,11 +32,11 @@ unique_ptr CreateBoundStructExtract(ClientContext &context, unique_p BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, bool root_expression) { // bind the children of the function expression if (depth > 0) { - return BindResult(binder.FormatError(function, "UNNEST() for correlated expressions is not supported yet")); + return BindResult(BinderException(function, "UNNEST() for correlated expressions is not supported yet")); } - string error; + ErrorData error; if (function.children.empty()) { - return BindResult(binder.FormatError(function, "UNNEST() requires a single argument")); + return BindResult(BinderException(function, "UNNEST() requires a single argument")); } idx_t max_depth = 1; if (function.children.size() != 1) { @@ -44,7 +44,7 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b bool supported_argument = false; for (idx_t i = 1; i < function.children.size(); i++) { if (has_parameter) { - return BindResult(binder.FormatError(function, "UNNEST() only supports a single additional argument")); + return BindResult(BinderException(function, "UNNEST() only supports a single additional argument")); } if (function.children[i]->HasParameter()) { throw ParameterNotAllowedException("Parameter not allowed in unnest parameter"); @@ -54,8 +54,8 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b } auto alias = function.children[i]->alias; BindChild(function.children[i], depth, error); - if (!error.empty()) { - return BindResult(error); + if (error.HasError()) { + return BindResult(std::move(error)); } auto &const_child = BoundExpression::GetExpression(*function.children[i]); auto value = ExpressionExecutor::EvaluateScalar(context, *const_child, true); @@ -78,13 +78,13 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b supported_argument = true; } if (!supported_argument) { - return BindResult(binder.FormatError(function, "UNNEST - unsupported extra argument, unnest only supports " - "recursive := [true/false] or max_depth := #")); + return BindResult(BinderException(function, "UNNEST - unsupported extra argument, unnest only supports " + "recursive := [true/false] or max_depth := #")); } } unnest_level++; BindChild(function.children[0], depth, error); - if (!error.empty()) { + if (error.HasError()) { // failed to bind // try to bind correlated columns manually auto result = BindCorrelatedColumns(function.children[0], error); @@ -100,6 +100,7 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b if (unnest_level > 0) { throw BinderException( + function, "Nested UNNEST calls are not supported - use UNNEST(x, recursive := true) to unnest multiple levels"); } @@ -111,7 +112,7 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b case LogicalTypeId::SQLNULL: break; default: - return BindResult(binder.FormatError(function, "UNNEST() can only be applied to lists, structs and NULL")); + return BindResult(BinderException(function, "UNNEST() can only be applied to lists, structs and NULL")); } idx_t list_unnests; @@ -138,7 +139,7 @@ BindResult SelectBinder::BindUnnest(FunctionExpression &function, idx_t depth, b } if (struct_unnests > 0 && !root_expression) { child = std::move(unnest_expr); - return BindResult(binder.FormatError( + return BindResult(BinderException( function, "UNNEST() on a struct column can only be applied as the root element of a SELECT expression")); } // perform all LIST unnests 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 610db526f..4dbd8625c 100644 --- a/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp +++ b/src/duckdb/src/planner/binder/expression/bind_window_expression.cpp @@ -1,4 +1,6 @@ #include "duckdb/parser/expression/window_expression.hpp" +#include "duckdb/parser/expression/constant_expression.hpp" +#include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/planner/expression/bound_aggregate_expression.hpp" #include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression/bound_columnref_expression.hpp" @@ -101,11 +103,11 @@ static LogicalType BindRangeExpression(ClientContext &context, const string &nam auto &bound = BoundExpression::GetExpression(*expr); children.emplace_back(std::move(bound)); - string error; + ErrorData error; FunctionBinder function_binder(context); auto function = function_binder.BindScalarFunction(DEFAULT_SCHEMA, name, std::move(children), error, true); if (!function) { - throw BinderException(error); + error.Throw(); } bound = std::move(function); return bound->return_type; @@ -114,23 +116,24 @@ static LogicalType BindRangeExpression(ClientContext &context, const string &nam BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { auto name = window.GetName(); - QueryErrorContext error_context(binder.GetRootStatement(), window.query_location); + QueryErrorContext error_context(window.query_location); if (inside_window) { - throw BinderException(error_context.FormatError("window function calls cannot be nested")); + throw BinderException(error_context, "window function calls cannot be nested"); } if (depth > 0) { - throw BinderException(error_context.FormatError("correlated columns in window functions not supported")); + throw BinderException(error_context, "correlated columns in window functions not supported"); } // If we have range expressions, then only one order by clause is allowed. - if ((window.start == WindowBoundary::EXPR_PRECEDING_RANGE || window.start == WindowBoundary::EXPR_FOLLOWING_RANGE || - window.end == WindowBoundary::EXPR_PRECEDING_RANGE || window.end == WindowBoundary::EXPR_FOLLOWING_RANGE) && - window.orders.size() != 1) { - throw BinderException(error_context.FormatError("RANGE frames must have only one ORDER BY expression")); + const auto is_range = + (window.start == WindowBoundary::EXPR_PRECEDING_RANGE || window.start == WindowBoundary::EXPR_FOLLOWING_RANGE || + window.end == WindowBoundary::EXPR_PRECEDING_RANGE || window.end == WindowBoundary::EXPR_FOLLOWING_RANGE); + if (is_range && window.orders.size() != 1) { + throw BinderException(error_context, "RANGE frames must have only one ORDER BY expression"); } // bind inside the children of the window function // we set the inside_window flag to true to prevent binding nested window functions this->inside_window = true; - string error; + ErrorData error; for (auto &child : window.children) { BindChild(child, depth, error); } @@ -139,6 +142,21 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { } for (auto &order : window.orders) { BindChild(order.expression, depth, error); + + // If the frame is a RANGE frame and the type is a time, + // then we have to convert the time to a timestamp to avoid wrapping. + if (!is_range) { + continue; + } + auto &order_expr = order.expression; + auto &bound_order = BoundExpression::GetExpression(*order_expr); + const auto type_id = bound_order->return_type.id(); + if (type_id == LogicalTypeId::TIME || type_id == LogicalTypeId::TIME_TZ) { + // Convert to time + epoch and rebind + unique_ptr epoch = make_uniq(Value::DATE(date_t::epoch())); + BindChild(epoch, depth, error); + BindRangeExpression(context, "+", order.expression, epoch); + } } BindChild(window.filter_expr, depth, error); BindChild(window.start_expr, depth, error); @@ -147,9 +165,9 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { BindChild(window.default_expr, depth, error); this->inside_window = false; - if (!error.empty()) { + if (error.HasError()) { // failed to bind children of window function - return BindResult(error); + return BindResult(std::move(error)); } // Restore any collation expressions @@ -197,11 +215,12 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { D_ASSERT(func.type == CatalogType::AGGREGATE_FUNCTION_ENTRY); // bind the aggregate - string error; + ErrorData error; FunctionBinder function_binder(context); auto best_function = function_binder.BindFunction(func.name, func.functions, types, error); if (best_function == DConstants::INVALID_INDEX) { - throw BinderException(binder.FormatError(window, error)); + error.AddQueryLocation(window); + error.Throw(); } // found a matching function! bind it as an aggregate auto bound_function = func.functions.GetFunctionByOffset(best_function); @@ -221,6 +240,7 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { result->partitions.push_back(GetExpression(child)); } result->ignore_nulls = window.ignore_nulls; + result->distinct = window.distinct; // Convert RANGE boundary expressions to ORDER +/- expressions. // Note that PRECEEDING and FOLLOWING refer to the sequential order in the frame, @@ -264,10 +284,10 @@ BindResult BaseSelectBinder::BindWindow(WindowExpression &window, idx_t depth) { auto &bound_order = BoundExpression::GetExpression(*order_expr); auto order_type = bound_order->return_type; if (window.start_expr) { - order_type = LogicalType::MaxLogicalType(order_type, start_type); + order_type = LogicalType::MaxLogicalType(context, order_type, start_type); } if (window.end_expr) { - order_type = LogicalType::MaxLogicalType(order_type, end_type); + order_type = LogicalType::MaxLogicalType(context, order_type, end_type); } // Cast all three to match diff --git a/src/duckdb/src/planner/binder/query_node/bind_cte_node.cpp b/src/duckdb/src/planner/binder/query_node/bind_cte_node.cpp index 42f14b960..bc8772a55 100644 --- a/src/duckdb/src/planner/binder/query_node/bind_cte_node.cpp +++ b/src/duckdb/src/planner/binder/query_node/bind_cte_node.cpp @@ -30,8 +30,19 @@ unique_ptr Binder::BindNode(CTENode &statement) { result->names[i] = statement.aliases[i]; } + // Rename columns if duplicate names are detected + idx_t index = 1; + vector names; + for (auto &n : result->names) { + string name = n; + while (find(names.begin(), names.end(), name) != names.end()) { + name = n + ":" + std::to_string(index++); + } + names.push_back(name); + } + // This allows the right side to reference the CTE - bind_context.AddGenericBinding(result->setop_index, statement.ctename, result->names, result->types); + bind_context.AddGenericBinding(result->setop_index, statement.ctename, names, result->types); result->child_binder = Binder::CreateBinder(context, this); @@ -43,17 +54,12 @@ unique_ptr Binder::BindNode(CTENode &statement) { statement.modifiers.clear(); // Add bindings of left side to temporary CTE bindings context - result->child_binder->bind_context.AddCTEBinding(result->setop_index, statement.ctename, result->names, - result->types); + result->child_binder->bind_context.AddCTEBinding(result->setop_index, statement.ctename, names, result->types); result->child = result->child_binder->BindNode(*statement.child); // the result types of the CTE are the types of the LHS result->types = result->child->types; - // names are picked from the LHS, unless aliases are explicitly specified result->names = result->child->names; - for (idx_t i = 0; i < statement.aliases.size() && i < result->names.size(); i++) { - result->names[i] = statement.aliases[i]; - } MoveCorrelatedExpressions(*result->query_binder); MoveCorrelatedExpressions(*result->child_binder); diff --git a/src/duckdb/src/planner/binder/query_node/bind_select_node.cpp b/src/duckdb/src/planner/binder/query_node/bind_select_node.cpp index a4906402b..41931f539 100644 --- a/src/duckdb/src/planner/binder/query_node/bind_select_node.cpp +++ b/src/duckdb/src/planner/binder/query_node/bind_select_node.cpp @@ -24,6 +24,7 @@ #include "duckdb/planner/expression_binder/where_binder.hpp" #include "duckdb/planner/expression_iterator.hpp" #include "duckdb/planner/query_node/bound_select_node.hpp" +#include "duckdb/parser/expression/function_expression.hpp" namespace duckdb { @@ -98,7 +99,7 @@ unique_ptr Binder::BindLimitPercent(OrderBinder &order_bind if (!result->limit) { result->limit_percent = val.IsNull() ? 100 : val.GetValue(); if (result->limit_percent < 0.0) { - throw Exception("Limit percentage can't be negative value"); + throw InvalidInputException("Limit percentage can't be negative value"); } } } @@ -158,6 +159,36 @@ void Binder::BindModifiers(OrderBinder &order_binder, QueryNode &statement, Boun order.orders = std::move(new_orders); } } +#if 0 + // When this verification is enabled, replace ORDER BY x, y with ORDER BY create_sort_key(x, y) + // note that we don't enable this during actual verification since it doesn't always work + // e.g. it breaks EXPLAIN output on queries + bool can_replace = true; + for (auto &order_node : order.orders) { + if (order_node.expression->type == ExpressionType::VALUE_CONSTANT) { + // we cannot replace the sort key when we order by literals (e.g. ORDER BY 1, 2` + can_replace = false; + break; + } + } + if (!order_binder.HasExtraList()) { + // we can only do the replacement when we can order by elements that are not in the selection list + can_replace = false; + } + if (can_replace) { + vector> sort_key_parameters; + for (auto &order_node : order.orders) { + sort_key_parameters.push_back(std::move(order_node.expression)); + auto type = config.ResolveOrder(order_node.type); + auto null_order = config.ResolveNullOrder(type, order_node.null_order); + string sort_param = EnumUtil::ToString(type) + " " + EnumUtil::ToString(null_order); + sort_key_parameters.push_back(make_uniq(Value(sort_param))); + } + order.orders.clear(); + auto create_sort_key = make_uniq("create_sort_key", std::move(sort_key_parameters)); + order.orders.emplace_back(OrderType::ASCENDING, OrderByNullType::NULLS_LAST, std::move(create_sort_key)); + } +#endif for (auto &order_node : order.orders) { vector> order_list; order_binders[0]->ExpandStarExpression(std::move(order_node.expression), order_list); @@ -184,7 +215,7 @@ void Binder::BindModifiers(OrderBinder &order_binder, QueryNode &statement, Boun bound_modifier = BindLimitPercent(order_binder, mod->Cast()); break; default: - throw Exception("Unsupported result modifier"); + throw InternalException("Unsupported result modifier"); } if (bound_modifier) { result.modifiers.push_back(std::move(bound_modifier)); @@ -562,13 +593,14 @@ unique_ptr Binder::BindSelectNode(SelectNode &statement, unique_ if (statement.aggregate_handling == AggregateHandling::FORCE_AGGREGATES) { error += "\nGROUP BY ALL will only group entries in the SELECT list. Add it to the SELECT list or " "GROUP BY this entry explicitly."; + throw BinderException(bound_columns[0].query_location, error, bound_columns[0].name); } else { error += "\nEither add it to the GROUP BY list, or use \"ANY_VALUE(%s)\" if the exact value of \"%s\" " "is not important."; + throw BinderException(bound_columns[0].query_location, error, bound_columns[0].name, + bound_columns[0].name, bound_columns[0].name); } - throw BinderException(FormatError(bound_columns[0].query_location, error, bound_columns[0].name, - bound_columns[0].name, bound_columns[0].name)); } } } 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 1150ea4c8..0d8d4445f 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 @@ -9,6 +9,7 @@ #include "duckdb/planner/expression_binder/order_binder.hpp" #include "duckdb/planner/query_node/bound_select_node.hpp" #include "duckdb/planner/query_node/bound_set_operation_node.hpp" +#include "duckdb/common/enum_util.hpp" namespace duckdb { @@ -71,37 +72,37 @@ static void GatherAliases(BoundQueryNode &node, case_insensitive_map_t &a } } -static void BuildUnionByNameInfo(BoundSetOperationNode &result, bool can_contain_nulls) { +static void BuildUnionByNameInfo(ClientContext &context, BoundSetOperationNode &result, bool can_contain_nulls) { D_ASSERT(result.setop_type == SetOperationType::UNION_BY_NAME); case_insensitive_map_t left_names_map; case_insensitive_map_t right_names_map; - BoundQueryNode *left_node = result.left.get(); - BoundQueryNode *right_node = result.right.get(); + auto &left_node = *result.left; + auto &right_node = *result.right; // Build a name_map to use to check if a name exists // We throw a binder exception if two same name in the SELECT list - for (idx_t i = 0; i < left_node->names.size(); ++i) { - if (left_names_map.find(left_node->names[i]) != left_names_map.end()) { + for (idx_t i = 0; i < left_node.names.size(); ++i) { + if (left_names_map.find(left_node.names[i]) != left_names_map.end()) { throw BinderException("UNION(ALL) BY NAME operation doesn't support same name in SELECT list"); } - left_names_map[left_node->names[i]] = i; + left_names_map[left_node.names[i]] = i; } - for (idx_t i = 0; i < right_node->names.size(); ++i) { - if (right_names_map.find(right_node->names[i]) != right_names_map.end()) { + for (idx_t i = 0; i < right_node.names.size(); ++i) { + if (right_names_map.find(right_node.names[i]) != right_names_map.end()) { throw BinderException("UNION(ALL) BY NAME operation doesn't support same name in SELECT list"); } - if (left_names_map.find(right_node->names[i]) == left_names_map.end()) { - result.names.push_back(right_node->names[i]); + if (left_names_map.find(right_node.names[i]) == left_names_map.end()) { + result.names.push_back(right_node.names[i]); } - right_names_map[right_node->names[i]] = i; + right_names_map[right_node.names[i]] = i; } idx_t new_size = result.names.size(); bool need_reorder = false; - vector left_reorder_idx(left_node->names.size()); - vector right_reorder_idx(right_node->names.size()); + vector left_reorder_idx(left_node.names.size()); + vector right_reorder_idx(right_node.names.size()); // Construct return type and reorder_idxs // reorder_idxs is used to gather correct alias_map @@ -113,20 +114,20 @@ static void BuildUnionByNameInfo(BoundSetOperationNode &result, bool can_contain bool right_exist = right_index != right_names_map.end(); LogicalType result_type; if (left_exist && right_exist) { - result_type = LogicalType::MaxLogicalType(left_node->types[left_index->second], - right_node->types[right_index->second]); + result_type = LogicalType::MaxLogicalType(context, left_node.types[left_index->second], + right_node.types[right_index->second]); if (left_index->second != i || right_index->second != i) { need_reorder = true; } left_reorder_idx[left_index->second] = i; right_reorder_idx[right_index->second] = i; } else if (left_exist) { - result_type = left_node->types[left_index->second]; + result_type = left_node.types[left_index->second]; need_reorder = true; left_reorder_idx[left_index->second] = i; } else { D_ASSERT(right_exist); - result_type = right_node->types[right_index->second]; + result_type = right_node.types[right_index->second]; need_reorder = true; right_reorder_idx[right_index->second] = i; } @@ -155,21 +156,21 @@ static void BuildUnionByNameInfo(BoundSetOperationNode &result, bool can_contain unique_ptr right_reorder_expr; if (left_exist && right_exist) { left_reorder_expr = make_uniq( - left_node->types[left_index->second], ColumnBinding(left_node->GetRootIndex(), left_index->second)); + left_node.types[left_index->second], ColumnBinding(left_node.GetRootIndex(), left_index->second)); right_reorder_expr = - make_uniq(right_node->types[right_index->second], - ColumnBinding(right_node->GetRootIndex(), right_index->second)); + make_uniq(right_node.types[right_index->second], + ColumnBinding(right_node.GetRootIndex(), right_index->second)); } else if (left_exist) { left_reorder_expr = make_uniq( - left_node->types[left_index->second], ColumnBinding(left_node->GetRootIndex(), left_index->second)); + left_node.types[left_index->second], ColumnBinding(left_node.GetRootIndex(), left_index->second)); // create null value here right_reorder_expr = make_uniq(Value(result.types[i])); } else { D_ASSERT(right_exist); left_reorder_expr = make_uniq(Value(result.types[i])); right_reorder_expr = - make_uniq(right_node->types[right_index->second], - ColumnBinding(right_node->GetRootIndex(), right_index->second)); + make_uniq(right_node.types[right_index->second], + ColumnBinding(right_node.GetRootIndex(), right_index->second)); } result.left_reorder_exprs.push_back(std::move(left_reorder_expr)); result.right_reorder_exprs.push_back(std::move(right_reorder_expr)); @@ -210,12 +211,11 @@ unique_ptr Binder::BindNode(SetOperationNode &statement) { } if (result->setop_type == SetOperationType::UNION_BY_NAME) { - BuildUnionByNameInfo(*result, can_contain_nulls); - + BuildUnionByNameInfo(context, *result, can_contain_nulls); } else { // figure out the types of the setop result by picking the max of both for (idx_t i = 0; i < result->left->types.size(); i++) { - auto result_type = LogicalType::MaxLogicalType(result->left->types[i], result->right->types[i]); + auto result_type = LogicalType::ForceMaxLogicalType(result->left->types[i], result->right->types[i]); if (!can_contain_nulls) { if (ExpressionBinder::ContainsNullType(result_type)) { result_type = ExpressionBinder::ExchangeNullType(result_type); diff --git a/src/duckdb/src/planner/binder/query_node/bind_table_macro_node.cpp b/src/duckdb/src/planner/binder/query_node/bind_table_macro_node.cpp index a3b7f58ca..4616fba53 100644 --- a/src/duckdb/src/planner/binder/query_node/bind_table_macro_node.cpp +++ b/src/duckdb/src/planner/binder/query_node/bind_table_macro_node.cpp @@ -32,7 +32,7 @@ unique_ptr Binder::BindTableMacro(FunctionExpression &function, Table if (!error.empty()) { // cannot use error below as binder rnot in scope // return BindResult(binder. FormatError(*expr->get(), error)); - throw BinderException(FormatError(function, error)); + throw BinderException(function, error); } // create a MacroBinding to bind this macro's parameters to its arguments @@ -58,10 +58,9 @@ unique_ptr Binder::BindTableMacro(FunctionExpression &function, Table auto eb = ExpressionBinder(*this, this->context); eb.macro_binding = new_macro_binding.get(); - - /* Does it all goes throu every expression in a selectstmt */ + vector> lambda_params; ParsedExpressionIterator::EnumerateQueryNodeChildren( - *node, [&](unique_ptr &child) { eb.ReplaceMacroParametersRecursive(child); }); + *node, [&](unique_ptr &child) { eb.ReplaceMacroParameters(child, lambda_params); }); return node; } diff --git a/src/duckdb/src/planner/binder/statement/bind_copy.cpp b/src/duckdb/src/planner/binder/statement/bind_copy.cpp index 251fb01e0..ac519a5dc 100644 --- a/src/duckdb/src/planner/binder/statement/bind_copy.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_copy.cpp @@ -5,7 +5,6 @@ #include "duckdb/common/bind_helpers.hpp" #include "duckdb/common/filename_pattern.hpp" #include "duckdb/common/local_file_system.hpp" -#include "duckdb/execution/operator/scan/csv/parallel_csv_reader.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/database.hpp" @@ -52,6 +51,10 @@ vector GetUniqueNames(const vector &original_names) { return unique_names; } +static bool GetBooleanArg(ClientContext &context, const vector &arg) { + return arg.empty() || arg[0].CastAs(context, LogicalType::BOOLEAN).GetValue(); +} + BoundStatement Binder::BindCopyTo(CopyStatement &stmt) { // COPY TO a file auto &config = DBConfig::GetConfig(context); @@ -76,59 +79,76 @@ BoundStatement Binder::BindCopyTo(CopyStatement &stmt) { if (!copy_function.function.copy_to_bind) { throw NotImplementedException("COPY TO is not supported for FORMAT \"%s\"", stmt.info->format); } + bool use_tmp_file = true; bool overwrite_or_ignore = false; FilenamePattern filename_pattern; bool user_set_use_tmp_file = false; bool per_thread_output = false; + optional_idx file_size_bytes; vector partition_cols; + CopyFunctionBindInput bind_input(*stmt.info); + + bind_input.file_extension = copy_function.function.extension; + auto original_options = stmt.info->options; stmt.info->options.clear(); for (auto &option : original_options) { auto loption = StringUtil::Lower(option.first); if (loption == "use_tmp_file") { - use_tmp_file = - option.second.empty() || option.second[0].CastAs(context, LogicalType::BOOLEAN).GetValue(); + use_tmp_file = GetBooleanArg(context, option.second); user_set_use_tmp_file = true; - continue; - } - if (loption == "overwrite_or_ignore") { - overwrite_or_ignore = - option.second.empty() || option.second[0].CastAs(context, LogicalType::BOOLEAN).GetValue(); - continue; - } - if (loption == "filename_pattern") { + } else if (loption == "overwrite_or_ignore") { + overwrite_or_ignore = GetBooleanArg(context, option.second); + } else if (loption == "filename_pattern") { if (option.second.empty()) { throw IOException("FILENAME_PATTERN cannot be empty"); } filename_pattern.SetFilenamePattern( option.second[0].CastAs(context, LogicalType::VARCHAR).GetValue()); - continue; - } - - if (loption == "per_thread_output") { - per_thread_output = - option.second.empty() || option.second[0].CastAs(context, LogicalType::BOOLEAN).GetValue(); - continue; - } - if (loption == "partition_by") { + } else if (loption == "file_extension") { + if (option.second.empty()) { + throw IOException("FILE_EXTENSION cannot be empty"); + } + bind_input.file_extension = option.second[0].CastAs(context, LogicalType::VARCHAR).GetValue(); + } else if (loption == "per_thread_output") { + per_thread_output = GetBooleanArg(context, option.second); + } else if (loption == "file_size_bytes") { + if (option.second.empty()) { + throw BinderException("FILE_SIZE_BYTES cannot be empty"); + } + if (!copy_function.function.file_size_bytes) { + throw NotImplementedException("FILE_SIZE_BYTES not implemented for FORMAT \"%s\"", stmt.info->format); + } + if (option.second[0].GetTypeMutable().id() == LogicalTypeId::VARCHAR) { + file_size_bytes = DBConfig::ParseMemoryLimit(option.second[0].ToString()); + } else { + file_size_bytes = option.second[0].GetValue(); + } + } else if (loption == "partition_by") { auto converted = ConvertVectorToValue(std::move(option.second)); partition_cols = ParseColumnsOrdered(converted, select_node.names, loption); - continue; + } else { + stmt.info->options[option.first] = option.second; } - stmt.info->options[option.first] = option.second; } if (user_set_use_tmp_file && per_thread_output) { throw NotImplementedException("Can't combine USE_TMP_FILE and PER_THREAD_OUTPUT for COPY"); } + if (user_set_use_tmp_file && file_size_bytes.IsValid()) { + throw NotImplementedException("Can't combine USE_TMP_FILE and FILE_SIZE_BYTES for COPY"); + } if (user_set_use_tmp_file && !partition_cols.empty()) { throw NotImplementedException("Can't combine USE_TMP_FILE and PARTITION_BY for COPY"); } if (per_thread_output && !partition_cols.empty()) { throw NotImplementedException("Can't combine PER_THREAD_OUTPUT and PARTITION_BY for COPY"); } + if (file_size_bytes.IsValid() && !partition_cols.empty()) { + throw NotImplementedException("Can't combine FILE_SIZE_BYTES and PARTITION_BY for COPY"); + } bool is_remote_file = config.file_system->IsRemoteFile(stmt.info->file_path); if (is_remote_file) { use_tmp_file = false; @@ -144,14 +164,19 @@ BoundStatement Binder::BindCopyTo(CopyStatement &stmt) { auto file_path = stmt.info->file_path; auto function_data = - copy_function.function.copy_to_bind(context, *stmt.info, unique_column_names, select_node.types); + copy_function.function.copy_to_bind(context, bind_input, unique_column_names, select_node.types); + // now create the copy information auto copy = make_uniq(copy_function.function, std::move(function_data), std::move(stmt.info)); copy->file_path = file_path; copy->use_tmp_file = use_tmp_file; copy->overwrite_or_ignore = overwrite_or_ignore; copy->filename_pattern = filename_pattern; + copy->file_extension = bind_input.file_extension; copy->per_thread_output = per_thread_output; + if (file_size_bytes.IsValid()) { + copy->file_size_bytes = file_size_bytes; + } copy->partition_output = !partition_cols.empty(); copy->partition_columns = std::move(partition_cols); diff --git a/src/duckdb/src/planner/binder/statement/bind_create.cpp b/src/duckdb/src/planner/binder/statement/bind_create.cpp index bafbdfc75..e0c7fc1c7 100644 --- a/src/duckdb/src/planner/binder/statement/bind_create.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_create.cpp @@ -3,6 +3,7 @@ #include "duckdb/catalog/catalog_entry/duck_table_entry.hpp" #include "duckdb/catalog/catalog_entry/schema_catalog_entry.hpp" #include "duckdb/catalog/catalog_entry/type_catalog_entry.hpp" +#include "duckdb/main/secret/secret_manager.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/database.hpp" #include "duckdb/parser/expression/constant_expression.hpp" @@ -14,6 +15,7 @@ #include "duckdb/parser/parsed_data/create_macro_info.hpp" #include "duckdb/parser/parsed_data/create_view_info.hpp" #include "duckdb/parser/tableref/table_function_ref.hpp" +#include "duckdb/parser/parsed_data/create_secret_info.hpp" #include "duckdb/parser/parsed_expression_iterator.hpp" #include "duckdb/parser/statement/create_statement.hpp" #include "duckdb/planner/binder.hpp" @@ -173,20 +175,20 @@ SchemaCatalogEntry &Binder::BindCreateFunctionInfo(CreateInfo &info) { } auto this_macro_binding = make_uniq(dummy_types, dummy_names, base.name); macro_binding = this_macro_binding.get(); - ExpressionBinder::QualifyColumnNames(*this, scalar_function.expression); // create a copy of the expression because we do not want to alter the original auto expression = scalar_function.expression->Copy(); + ExpressionBinder::QualifyColumnNames(*this, expression); // bind it to verify the function was defined correctly - string error; + ErrorData error; auto sel_node = make_uniq(); auto group_info = make_uniq(); SelectBinder binder(*this, context, *sel_node, *group_info); error = binder.Bind(expression, 0, false); - if (!error.empty()) { - throw BinderException(error); + if (error.HasError()) { + error.Throw(); } return BindCreateSchema(info); @@ -245,7 +247,7 @@ void Binder::BindLogicalType(ClientContext &context, LogicalType &type, optional } if (type.id() == LogicalTypeId::INVALID) { - type = Catalog::GetType(context, INVALID_CATALOG, schema, user_type_name); + type = Catalog::GetType(context, INVALID_CATALOG, INVALID_SCHEMA, user_type_name); } } else { string type_catalog = UserType::GetCatalog(type); @@ -632,21 +634,23 @@ BoundStatement Binder::Bind(CreateStatement &stmt) { } result.plan->AddChild(std::move(query)); - } else if (create_type_info.type.id() == LogicalTypeId::USER) { + } else { // two cases: - // 1: create a type with a non-existant type as source, catalog.GetType(...) will throw exception. + // 1: create a type with a non-existent type as source, Binder::BindLogicalType(...) will throw exception. // 2: create a type alias with a custom type. // eg. CREATE TYPE a AS INT; CREATE TYPE b AS a; // We set b to be an alias for the underlying type of a - auto inner_type = Catalog::GetType(context, schema.catalog.GetName(), schema.name, - UserType::GetTypeName(create_type_info.type)); - inner_type.SetAlias(create_type_info.name); - create_type_info.type = inner_type; + Binder::BindLogicalType(context, create_type_info.type); } break; } + case CatalogType::SECRET_ENTRY: { + CatalogTransaction transaction = CatalogTransaction(Catalog::GetSystemCatalog(context), context); + properties.return_type = StatementReturnType::QUERY_RESULT; + return SecretManager::Get(context).BindCreateSecret(transaction, stmt.info->Cast()); + } default: - throw Exception("Unrecognized type!"); + throw InternalException("Unrecognized type!"); } properties.return_type = StatementReturnType::NOTHING; properties.allow_stream_result = false; diff --git a/src/duckdb/src/planner/binder/statement/bind_create_table.cpp b/src/duckdb/src/planner/binder/statement/bind_create_table.cpp index 56672a22e..683c475dd 100644 --- a/src/duckdb/src/planner/binder/statement/bind_create_table.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_create_table.cpp @@ -171,9 +171,9 @@ void Binder::BindGeneratedColumns(BoundCreateTableInfo &info) { auto binder = Binder::CreateBinder(context); binder->bind_context.AddGenericBinding(table_index, base.table, names, types); auto expr_binder = ExpressionBinder(*binder, context); - string ignore; + ErrorData ignore; auto table_binding = binder->bind_context.GetBinding(base.table, ignore); - D_ASSERT(table_binding && ignore.empty()); + D_ASSERT(table_binding && !ignore.HasError()); auto bind_order = info.column_dependency_manager.GetBindOrder(base.columns); logical_index_set_t bound_indices; @@ -216,6 +216,9 @@ void Binder::BindDefaultValues(const ColumnList &columns, vectorHasParameter()) { + throw BinderException("DEFAULT values cannot contain parameters"); + } ConstantBinder default_binder(*this, context, "DEFAULT value"); default_binder.target_type = column.Type(); bound_default = default_binder.Bind(default_copy); diff --git a/src/duckdb/src/planner/binder/statement/bind_drop.cpp b/src/duckdb/src/planner/binder/statement/bind_drop.cpp index 056c70b6d..65baae2ef 100644 --- a/src/duckdb/src/planner/binder/statement/bind_drop.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_drop.cpp @@ -50,8 +50,13 @@ BoundStatement Binder::Bind(DropStatement &stmt) { stmt.info->schema = entry->ParentSchema().name; break; } + case CatalogType::SECRET_ENTRY: { + //! Secrets are stored in the secret manager; they can always be dropped + properties.requires_valid_transaction = false; + break; + } default: - throw BinderException("Unknown catalog type for drop statement!"); + throw BinderException("Unknown catalog type for drop statement: '%s'", CatalogTypeToString(stmt.info->type)); } result.plan = make_uniq(LogicalOperatorType::LOGICAL_DROP, std::move(stmt.info)); result.names = {"Success"}; diff --git a/src/duckdb/src/planner/binder/statement/bind_insert.cpp b/src/duckdb/src/planner/binder/statement/bind_insert.cpp index 337f1df66..d8a252af0 100644 --- a/src/duckdb/src/planner/binder/statement/bind_insert.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_insert.cpp @@ -324,9 +324,9 @@ void Binder::BindOnConflictClause(LogicalInsert &insert, TableCatalogEntry &tabl throw InternalException("Could not locate a table_index from the children of the insert"); } - string unused; + ErrorData unused; auto original_binding = bind_context.GetBinding(table_alias, unused); - D_ASSERT(original_binding); + D_ASSERT(original_binding && !unused.HasError()); auto table_index = original_binding->index; diff --git a/src/duckdb/src/planner/binder/statement/bind_pragma.cpp b/src/duckdb/src/planner/binder/statement/bind_pragma.cpp index 08603fd91..12f30e92c 100644 --- a/src/duckdb/src/planner/binder/statement/bind_pragma.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_pragma.cpp @@ -30,10 +30,12 @@ unique_ptr Binder::BindPragma(PragmaInfo &info, QueryErrorConte // bind the pragma function auto &entry = Catalog::GetEntry(context, INVALID_CATALOG, DEFAULT_SCHEMA, info.name); FunctionBinder function_binder(context); - string error; + ErrorData error; idx_t bound_idx = function_binder.BindFunction(entry.name, entry.functions, params, error); if (bound_idx == DConstants::INVALID_INDEX) { - throw BinderException(error_context.FormatError(error)); + D_ASSERT(error.HasError()); + error.AddQueryLocation(error_context); + error.Throw(); } auto bound_function = entry.functions.GetFunctionByOffset(bound_idx); // bind and check named params @@ -43,7 +45,7 @@ unique_ptr Binder::BindPragma(PragmaInfo &info, QueryErrorConte BoundStatement Binder::Bind(PragmaStatement &stmt) { // bind the pragma function - QueryErrorContext error_context(root_statement, stmt.stmt_location); + QueryErrorContext error_context(stmt.stmt_location); auto bound_info = BindPragma(*stmt.info, error_context); if (!bound_info->function.function) { throw BinderException("PRAGMA function does not have a function specified"); diff --git a/src/duckdb/src/planner/binder/statement/bind_show.cpp b/src/duckdb/src/planner/binder/statement/bind_show.cpp deleted file mode 100644 index a6a50e2b3..000000000 --- a/src/duckdb/src/planner/binder/statement/bind_show.cpp +++ /dev/null @@ -1,30 +0,0 @@ -#include "duckdb/planner/binder.hpp" -#include "duckdb/parser/statement/show_statement.hpp" -#include "duckdb/planner/operator/logical_show.hpp" - -namespace duckdb { - -BoundStatement Binder::Bind(ShowStatement &stmt) { - BoundStatement result; - - if (stmt.info->is_summary) { - return BindSummarize(stmt); - } - auto plan = Bind(*stmt.info->query); - stmt.info->types = plan.types; - stmt.info->aliases = plan.names; - - auto show = make_uniq(std::move(plan.plan)); - show->types_select = plan.types; - show->aliases = plan.names; - - result.plan = std::move(show); - - result.names = {"column_name", "column_type", "null", "key", "default", "extra"}; - result.types = {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR, - LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}; - properties.return_type = StatementReturnType::QUERY_RESULT; - return result; -} - -} // namespace duckdb diff --git a/src/duckdb/src/planner/binder/statement/bind_simple.cpp b/src/duckdb/src/planner/binder/statement/bind_simple.cpp index 30907a54c..3d1b73f01 100644 --- a/src/duckdb/src/planner/binder/statement/bind_simple.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_simple.cpp @@ -17,6 +17,7 @@ BoundStatement Binder::Bind(AlterStatement &stmt) { result.names = {"Success"}; result.types = {LogicalType::BOOLEAN}; BindSchemaOrCatalog(stmt.info->catalog, stmt.info->schema); + auto entry = Catalog::GetEntry(context, stmt.info->GetCatalogType(), stmt.info->catalog, stmt.info->schema, stmt.info->name, stmt.info->if_not_found); if (entry) { diff --git a/src/duckdb/src/planner/binder/statement/bind_summarize.cpp b/src/duckdb/src/planner/binder/statement/bind_summarize.cpp index 08df27ca8..e8eb87f3e 100644 --- a/src/duckdb/src/planner/binder/statement/bind_summarize.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_summarize.cpp @@ -1,11 +1,13 @@ #include "duckdb/planner/binder.hpp" -#include "duckdb/parser/statement/show_statement.hpp" -#include "duckdb/planner/operator/logical_show.hpp" #include "duckdb/parser/query_node/select_node.hpp" #include "duckdb/parser/expression/function_expression.hpp" #include "duckdb/parser/expression/constant_expression.hpp" #include "duckdb/parser/expression/cast_expression.hpp" #include "duckdb/parser/tableref/subqueryref.hpp" +#include "duckdb/parser/tableref/showref.hpp" +#include "duckdb/parser/tableref/basetableref.hpp" +#include "duckdb/parser/expression/star_expression.hpp" +#include "duckdb/planner/bound_tableref.hpp" namespace duckdb { @@ -65,12 +67,26 @@ static unique_ptr SummarizeCreateNullPercentage(string column_ return make_uniq(LogicalType::DECIMAL(9, 2), std::move(percentage_x)); } -BoundStatement Binder::BindSummarize(ShowStatement &stmt) { - auto query_copy = stmt.info->query->Copy(); +unique_ptr Binder::BindSummarize(ShowRef &ref) { + unique_ptr query; + if (ref.query) { + query = std::move(ref.query); + } else { + auto table_name = QualifiedName::Parse(ref.table_name); + auto node = make_uniq(); + node->select_list.push_back(make_uniq()); + auto basetableref = make_uniq(); + basetableref->catalog_name = table_name.catalog; + basetableref->schema_name = table_name.schema; + basetableref->table_name = table_name.name; + node->from_table = std::move(basetableref); + query = std::move(node); + } + auto query_copy = query->Copy(); // we bind the plan once in a child-node to figure out the column names and column types - auto child_binder = Binder::CreateBinder(context); - auto plan = child_binder->Bind(*stmt.info->query); + auto child_binder = Binder::CreateBinder(context, this); + auto plan = child_binder->Bind(*query); D_ASSERT(plan.types.size() == plan.names.size()); vector> name_children; vector> type_children; @@ -127,8 +143,10 @@ BoundStatement Binder::BindSummarize(ShowStatement &stmt) { select_node->select_list.push_back(SummarizeWrapUnnest(null_percentage_children, "null_percentage")); select_node->from_table = std::move(subquery_ref); - properties.return_type = StatementReturnType::QUERY_RESULT; - return Bind(*select_node); + auto select_stmt = make_uniq(); + select_stmt->node = std::move(select_node); + auto subquery = make_uniq(std::move(select_stmt)); + return Bind(*subquery); } } // namespace duckdb diff --git a/src/duckdb/src/planner/binder/statement/bind_vacuum.cpp b/src/duckdb/src/planner/binder/statement/bind_vacuum.cpp index fc36d55a2..c7aaa0abb 100644 --- a/src/duckdb/src/planner/binder/statement/bind_vacuum.cpp +++ b/src/duckdb/src/planner/binder/statement/bind_vacuum.cpp @@ -51,7 +51,7 @@ BoundStatement Binder::Bind(VacuumStatement &stmt) { ColumnRefExpression colref(col_name, table.name); auto result = bind_context.BindColumn(colref, 0); if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } select_list.push_back(std::move(result.expression)); } diff --git a/src/duckdb/src/planner/binder/tableref/bind_basetableref.cpp b/src/duckdb/src/planner/binder/tableref/bind_basetableref.cpp index 3a4a0d6d5..2300fe8c7 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_basetableref.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_basetableref.cpp @@ -75,7 +75,7 @@ unique_ptr Binder::BindWithReplacementScan(ClientContext &context } unique_ptr Binder::Bind(BaseTableRef &ref) { - QueryErrorContext error_context(root_statement, ref.query_location); + QueryErrorContext error_context(ref.query_location); // CTEs and views are also referred to using BaseTableRefs, hence need to distinguish here // check if the table name refers to a CTE diff --git a/src/duckdb/src/planner/binder/tableref/bind_expressionlistref.cpp b/src/duckdb/src/planner/binder/tableref/bind_expressionlistref.cpp index a506764b2..7176fb682 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_expressionlistref.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_expressionlistref.cpp @@ -11,6 +11,7 @@ unique_ptr Binder::Bind(ExpressionListRef &expr) { auto result = make_uniq(); result->types = expr.expected_types; result->names = expr.expected_names; + auto prev_can_contain_nulls = this->can_contain_nulls; // bind value list InsertBinder binder(*this, context); binder.target_type = LogicalType(LogicalTypeId::INVALID); @@ -23,16 +24,18 @@ unique_ptr Binder::Bind(ExpressionListRef &expr) { } } + this->can_contain_nulls = true; vector> list; for (idx_t val_idx = 0; val_idx < expression_list.size(); val_idx++) { if (!result->types.empty()) { D_ASSERT(result->types.size() == expression_list.size()); binder.target_type = result->types[val_idx]; } - auto expr = binder.Bind(expression_list[val_idx]); - list.push_back(std::move(expr)); + auto bound_expr = binder.Bind(expression_list[val_idx]); + list.push_back(std::move(bound_expr)); } result->values.push_back(std::move(list)); + this->can_contain_nulls = prev_can_contain_nulls; } if (result->types.empty() && !expr.values.empty()) { // there are no types specified @@ -44,10 +47,14 @@ unique_ptr Binder::Bind(ExpressionListRef &expr) { for (idx_t list_idx = 0; list_idx < result->values.size(); list_idx++) { auto &list = result->values[list_idx]; for (idx_t val_idx = 0; val_idx < list.size(); val_idx++) { - result->types[val_idx] = - LogicalType::MaxLogicalType(result->types[val_idx], list[val_idx]->return_type); + auto ¤t_type = result->types[val_idx]; + auto next_type = ExpressionBinder::GetExpressionReturnType(*list[val_idx]); + result->types[val_idx] = LogicalType::MaxLogicalType(context, current_type, next_type); } } + for (auto &type : result->types) { + type = LogicalType::NormalizeType(type); + } // finally do another loop over the expressions and add casts where required for (idx_t list_idx = 0; list_idx < result->values.size(); list_idx++) { auto &list = result->values[list_idx]; diff --git a/src/duckdb/src/planner/binder/tableref/bind_joinref.cpp b/src/duckdb/src/planner/binder/tableref/bind_joinref.cpp index a8a0c9ede..1de85aa36 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_joinref.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_joinref.cpp @@ -205,7 +205,7 @@ unique_ptr Binder::Bind(JoinRef &ref) { } error_msg += "\n Left candidates: " + left_candidates; error_msg += "\n Right candidates: " + right_candidates; - throw BinderException(FormatError(ref, error_msg)); + throw BinderException(ref, error_msg); } break; } diff --git a/src/duckdb/src/planner/binder/tableref/bind_named_parameters.cpp b/src/duckdb/src/planner/binder/tableref/bind_named_parameters.cpp index e7d9fa1e2..e2a6537d1 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_named_parameters.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_named_parameters.cpp @@ -22,8 +22,8 @@ void Binder::BindNamedParameters(named_parameter_type_map_t &types, named_parame } else { error_msg = "Candidates:\n" + named_params; } - throw BinderException(error_context.FormatError("Invalid named parameter \"%s\" for function %s\n%s", - kv.first, func_name, error_msg)); + throw BinderException(error_context, "Invalid named parameter \"%s\" for function %s\n%s", kv.first, + func_name, error_msg); } if (entry->second.id() != LogicalTypeId::ANY) { kv.second = kv.second.DefaultCastAs(entry->second); diff --git a/src/duckdb/src/planner/binder/tableref/bind_pivot.cpp b/src/duckdb/src/planner/binder/tableref/bind_pivot.cpp index ed474f243..3ca1a8879 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_pivot.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_pivot.cpp @@ -108,7 +108,8 @@ static unique_ptr PivotFilteredAggregate(PivotRef &ref, vector(LogicalType::VARCHAR, pivot_expr->Copy()); - auto constant_value = make_uniq(pivot_value.values[pivot_value_idx++]); + auto constant_value = make_uniq( + pivot_value.values[pivot_value_idx++].DefaultCastAs(LogicalType::VARCHAR)); auto comp_expr = make_uniq(ExpressionType::COMPARE_NOT_DISTINCT_FROM, std::move(column_ref), std::move(constant_value)); if (filter) { @@ -355,13 +356,13 @@ unique_ptr Binder::BindPivot(PivotRef &ref, vectortype != ExpressionType::FUNCTION) { - throw BinderException(FormatError(*aggr, "Pivot expression must be an aggregate")); + throw BinderException(*aggr, "Pivot expression must be an aggregate"); } if (aggr->HasSubquery()) { - throw BinderException(FormatError(*aggr, "Pivot expression cannot contain subqueries")); + throw BinderException(*aggr, "Pivot expression cannot contain subqueries"); } if (aggr->IsWindow()) { - throw BinderException(FormatError(*aggr, "Pivot expression cannot contain window functions")); + throw BinderException(*aggr, "Pivot expression cannot contain window functions"); } // bind the function as an aggregate to ensure it is an aggregate and not a scalar function auto &aggr_function = aggr->Cast(); @@ -376,9 +377,8 @@ unique_ptr Binder::BindPivot(PivotRef &ref, vector Binder::BindPivot(PivotRef &ref, vector Binder::BindPivot(PivotRef &ref, vector= pivot_limit) { - throw BinderException("Pivot column limit of %llu exceeded. Use SET pivot_limit=X to increase the limit.", + throw BinderException(ref, "Pivot column limit of %llu exceeded. Use SET pivot_limit=X to increase the limit.", client_config.pivot_limit); } @@ -527,8 +526,8 @@ unique_ptr Binder::BindUnpivot(Binder &child_binder, PivotRef &ref, } if (!handled_columns.empty()) { for (auto &entry : handled_columns) { - throw BinderException("Column \"%s\" referenced in UNPIVOT but no matching entry was found in the table", - entry); + throw BinderException( + ref, "Column \"%s\" referenced in UNPIVOT but no matching entry was found in the table", entry); } } vector unpivot_names; @@ -550,6 +549,7 @@ unique_ptr Binder::BindUnpivot(Binder &child_binder, PivotRef &ref, for (idx_t v_idx = 1; v_idx < unpivot.entries.size(); v_idx++) { if (unpivot.entries[v_idx].values.size() != unpivot.entries[0].values.size()) { throw BinderException( + ref, "UNPIVOT value count mismatch - entry has %llu values, but expected all entries to have %llu values", unpivot.entries[v_idx].values.size(), unpivot.entries[0].values.size()); } @@ -559,7 +559,8 @@ unique_ptr Binder::BindUnpivot(Binder &child_binder, PivotRef &ref, vector> expressions; expressions.reserve(unpivot.entries.size()); for (auto &entry : unpivot.entries) { - expressions.push_back(make_uniq(entry.values[v_idx].ToString())); + auto colref = make_uniq(entry.values[v_idx].ToString()); + expressions.push_back(std::move(colref)); } unpivot_expressions.push_back(std::move(expressions)); } @@ -575,8 +576,8 @@ unique_ptr Binder::BindUnpivot(Binder &child_binder, PivotRef &ref, // construct the UNNEST expression for the set of unpivoted columns if (ref.unpivot_names.size() != unpivot_expressions.size()) { - throw BinderException("UNPIVOT name count mismatch - got %d names but %d expressions", ref.unpivot_names.size(), - unpivot_expressions.size()); + throw BinderException(ref, "UNPIVOT name count mismatch - got %d names but %d expressions", + ref.unpivot_names.size(), unpivot_expressions.size()); } for (idx_t i = 0; i < unpivot_expressions.size(); i++) { auto list_expr = make_uniq("list_value", std::move(unpivot_expressions[i])); diff --git a/src/duckdb/src/planner/binder/tableref/bind_showref.cpp b/src/duckdb/src/planner/binder/tableref/bind_showref.cpp new file mode 100644 index 000000000..d804122a7 --- /dev/null +++ b/src/duckdb/src/planner/binder/tableref/bind_showref.cpp @@ -0,0 +1,85 @@ +#include "duckdb/planner/binder.hpp" +#include "duckdb/parser/tableref/showref.hpp" +#include "duckdb/planner/tableref/bound_table_function.hpp" +#include "duckdb/planner/operator/logical_column_data_get.hpp" +#include "duckdb/function/pragma/pragma_functions.hpp" +#include "duckdb/parser/parsed_data/create_view_info.hpp" +#include "duckdb/parser/tableref/subqueryref.hpp" + +namespace duckdb { + +unique_ptr Binder::BindShowQuery(ShowRef &ref) { + // bind the child plan of the DESCRIBE statement + auto child_binder = Binder::CreateBinder(context, this); + auto plan = child_binder->Bind(*ref.query); + + // construct a column data collection with the result + vector return_names = {"column_name", "column_type", "null", "key", "default", "extra"}; + vector return_types = {LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR, + LogicalType::VARCHAR, LogicalType::VARCHAR, LogicalType::VARCHAR}; + DataChunk output; + output.Initialize(Allocator::Get(context), return_types); + + auto collection = make_uniq(context, return_types); + ColumnDataAppendState append_state; + collection->InitializeAppend(append_state); + for (idx_t column_idx = 0; column_idx < plan.types.size(); column_idx++) { + auto type = plan.types[column_idx]; + auto &name = plan.names[column_idx]; + + // "name", TypeId::VARCHAR + output.SetValue(0, output.size(), Value(name)); + // "type", TypeId::VARCHAR + output.SetValue(1, output.size(), Value(type.ToString())); + // "null", TypeId::VARCHAR + output.SetValue(2, output.size(), Value("YES")); + // "pk", TypeId::BOOL + output.SetValue(3, output.size(), Value()); + // "dflt_value", TypeId::VARCHAR + output.SetValue(4, output.size(), Value()); + // "extra", TypeId::VARCHAR + output.SetValue(5, output.size(), Value()); + + output.SetCardinality(output.size() + 1); + if (output.size() == STANDARD_VECTOR_SIZE) { + collection->Append(append_state, output); + output.Reset(); + } + } + collection->Append(append_state, output); + + auto show = make_uniq(GenerateTableIndex(), return_types, std::move(collection)); + bind_context.AddGenericBinding(show->table_index, "__show_select", return_names, return_types); + return make_uniq(std::move(show)); +} + +unique_ptr Binder::BindShowTable(ShowRef &ref) { + auto lname = StringUtil::Lower(ref.table_name); + + string sql; + if (lname == "\"databases\"") { + sql = PragmaShowDatabases(); + } else if (lname == "\"tables\"") { + sql = PragmaShowTables(); + } else if (lname == "__show_tables_expanded") { + sql = PragmaShowTablesExpanded(); + } else { + sql = PragmaShow(ref.table_name); + } + auto select = CreateViewInfo::ParseSelect(sql); + auto subquery = make_uniq(std::move(select)); + return Bind(*subquery); +} + +unique_ptr Binder::Bind(ShowRef &ref) { + if (ref.show_type == ShowType::SUMMARY) { + return BindSummarize(ref); + } + if (ref.query) { + return BindShowQuery(ref); + } else { + return BindShowTable(ref); + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/planner/binder/tableref/bind_table_function.cpp b/src/duckdb/src/planner/binder/tableref/bind_table_function.cpp index a3eddeaca..259624320 100644 --- a/src/duckdb/src/planner/binder/tableref/bind_table_function.cpp +++ b/src/duckdb/src/planner/binder/tableref/bind_table_function.cpp @@ -30,7 +30,7 @@ static bool IsTableInTableOutFunction(TableFunctionCatalogEntry &table_function) } bool Binder::BindTableInTableOutFunction(vector> &expressions, - unique_ptr &subquery, string &error) { + unique_ptr &subquery, ErrorData &error) { auto binder = Binder::CreateBinder(this->context, this, true); unique_ptr subquery_node; if (expressions.size() == 1 && expressions[0]->type == ExpressionType::SUBQUERY) { @@ -55,7 +55,7 @@ bool Binder::BindTableFunctionParameters(TableFunctionCatalogEntry &table_functi vector> &expressions, vector &arguments, vector ¶meters, named_parameter_map_t &named_parameters, - unique_ptr &subquery, string &error) { + unique_ptr &subquery, ErrorData &error) { if (IsTableInTableOutFunction(table_function)) { // special case binding for table-in table-out function arguments.emplace_back(LogicalTypeId::TABLE); @@ -88,7 +88,7 @@ bool Binder::BindTableFunctionParameters(TableFunctionCatalogEntry &table_functi // this separate subquery binding path is only used by python_map // FIXME: this should be unified with `BindTableInTableOutFunction` above if (seen_subquery) { - error = "Table function can have at most one subquery parameter "; + error = ErrorData("Table function can have at most one subquery parameter"); return false; } auto binder = Binder::CreateBinder(this->context, this, true); @@ -116,10 +116,10 @@ bool Binder::BindTableFunctionParameters(TableFunctionCatalogEntry &table_functi if (parameter_name.empty()) { // unnamed parameter if (!named_parameters.empty()) { - error = "Unnamed parameters cannot come after named parameters"; + error = ErrorData("Unnamed parameters cannot come after named parameters"); return false; } - arguments.emplace_back(sql_type); + arguments.emplace_back(constant.IsNull() ? LogicalType::SQLNULL : sql_type); parameters.emplace_back(std::move(constant)); } else { named_parameters[parameter_name] = std::move(constant); @@ -155,14 +155,6 @@ Binder::BindTableFunctionInternal(TableFunction &table_function, const string &f auto &arrow_bind = bind_data->Cast(); arrow_bind.external_dependency = std::move(external_dependency); } - if (table_function.name == "read_csv" || table_function.name == "read_csv_auto") { - auto &csv_bind = bind_data->Cast(); - if (csv_bind.single_threaded) { - table_function.extra_info = "(Single-Threaded)"; - } else { - table_function.extra_info = "(Multi-Threaded)"; - } - } } else { throw InvalidInputException("Cannot call function \"%s\" directly - it has no bind function", table_function.name); @@ -212,7 +204,7 @@ unique_ptr Binder::BindTableFunction(TableFunction &function, v } unique_ptr Binder::Bind(TableFunctionRef &ref) { - QueryErrorContext error_context(root_statement, ref.query_location); + QueryErrorContext error_context(ref.query_location); D_ASSERT(ref.function->type == ExpressionType::FUNCTION); auto &fexpr = ref.function->Cast(); @@ -250,17 +242,19 @@ unique_ptr Binder::Bind(TableFunctionRef &ref) { vector parameters; named_parameter_map_t named_parameters; unique_ptr subquery; - string error; + ErrorData error; if (!BindTableFunctionParameters(function, fexpr.children, arguments, parameters, named_parameters, subquery, error)) { - throw BinderException(FormatError(ref, error)); + error.AddQueryLocation(ref); + error.Throw(); } // select the function based on the input parameters FunctionBinder function_binder(context); idx_t best_function_idx = function_binder.BindFunction(function.name, function.functions, arguments, error); if (best_function_idx == DConstants::INVALID_INDEX) { - throw BinderException(FormatError(ref, error)); + error.AddQueryLocation(ref); + error.Throw(); } auto table_function = function.functions.GetFunctionByOffset(best_function_idx); diff --git a/src/duckdb/src/planner/bound_result_modifier.cpp b/src/duckdb/src/planner/bound_result_modifier.cpp index 0518eb7b1..dea41088a 100644 --- a/src/duckdb/src/planner/bound_result_modifier.cpp +++ b/src/duckdb/src/planner/bound_result_modifier.cpp @@ -1,4 +1,5 @@ #include "duckdb/planner/bound_result_modifier.hpp" +#include "duckdb/parser/expression_map.hpp" namespace duckdb { @@ -92,6 +93,29 @@ bool BoundOrderModifier::Equals(const unique_ptr &left, return BoundOrderModifier::Equals(*left, *right); } +bool BoundOrderModifier::Simplify(const vector> &groups) { + // for each ORDER BY - check if it is actually necessary + // expressions that are in the groups do not need to be ORDERED BY + // `ORDER BY` on a group has no effect, because for each aggregate, the group is unique + // similarly, we only need to ORDER BY each aggregate once + expression_set_t seen_expressions; + for (auto &target : groups) { + seen_expressions.insert(*target); + } + vector new_order_nodes; + for (auto &order_node : orders) { + if (seen_expressions.find(*order_node.expression) != seen_expressions.end()) { + // we do not need to order by this node + continue; + } + seen_expressions.insert(*order_node.expression); + new_order_nodes.push_back(std::move(order_node)); + } + orders.swap(new_order_nodes); + + return orders.empty(); +} + BoundLimitModifier::BoundLimitModifier() : BoundResultModifier(ResultModifierType::LIMIT_MODIFIER) { } diff --git a/src/duckdb/src/planner/expression.cpp b/src/duckdb/src/planner/expression.cpp index 2d6487712..9fa426b8a 100644 --- a/src/duckdb/src/planner/expression.cpp +++ b/src/duckdb/src/planner/expression.cpp @@ -38,14 +38,24 @@ bool Expression::IsScalar() const { return is_scalar; } -bool Expression::HasSideEffects() const { - bool has_side_effects = false; +bool Expression::IsVolatile() const { + bool is_volatile = false; ExpressionIterator::EnumerateChildren(*this, [&](const Expression &child) { - if (child.HasSideEffects()) { - has_side_effects = true; + if (child.IsVolatile()) { + is_volatile = true; } }); - return has_side_effects; + return is_volatile; +} + +bool Expression::IsConsistent() const { + bool is_consistent = true; + ExpressionIterator::EnumerateChildren(*this, [&](const Expression &child) { + if (!child.IsConsistent()) { + is_consistent = false; + } + }); + return is_consistent; } bool Expression::PropagatesNullValues() const { diff --git a/src/duckdb/src/planner/expression/bound_cast_expression.cpp b/src/duckdb/src/planner/expression/bound_cast_expression.cpp index 1eca9b3f4..9699fd706 100644 --- a/src/duckdb/src/planner/expression/bound_cast_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_cast_expression.cpp @@ -1,6 +1,7 @@ #include "duckdb/planner/expression/bound_cast_expression.hpp" #include "duckdb/planner/expression/bound_default_expression.hpp" #include "duckdb/planner/expression/bound_parameter_expression.hpp" +#include "duckdb/planner/expression/bound_constant_expression.hpp" #include "duckdb/function/cast_rules.hpp" #include "duckdb/function/cast/cast_function_set.hpp" #include "duckdb/main/config.hpp" @@ -28,7 +29,7 @@ BoundCastExpression::BoundCastExpression(ClientContext &context, unique_ptr AddCastExpressionInternal(unique_ptr expr, const LogicalType &target_type, BoundCastInfo bound_cast, bool try_cast) { - if (expr->return_type == target_type) { + if (ExpressionBinder::GetExpressionReturnType(*expr) == target_type) { return expr; } auto &expr_type = expr->return_type; diff --git a/src/duckdb/src/planner/expression/bound_function_expression.cpp b/src/duckdb/src/planner/expression/bound_function_expression.cpp index 5c3517b5a..2197b6764 100644 --- a/src/duckdb/src/planner/expression/bound_function_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_function_expression.cpp @@ -18,8 +18,12 @@ BoundFunctionExpression::BoundFunctionExpression(LogicalType return_type, Scalar D_ASSERT(!function.name.empty()); } -bool BoundFunctionExpression::HasSideEffects() const { - return function.side_effects == FunctionSideEffects::HAS_SIDE_EFFECTS ? true : Expression::HasSideEffects(); +bool BoundFunctionExpression::IsVolatile() const { + return function.stability == FunctionStability::VOLATILE ? true : Expression::IsVolatile(); +} + +bool BoundFunctionExpression::IsConsistent() const { + return function.stability != FunctionStability::CONSISTENT ? false : Expression::IsConsistent(); } bool BoundFunctionExpression::IsFoldable() const { @@ -30,12 +34,12 @@ bool BoundFunctionExpression::IsFoldable() const { auto &lambda_bind_data = bind_info->Cast(); if (lambda_bind_data.lambda_expr) { auto &expr = *lambda_bind_data.lambda_expr; - if (expr.HasSideEffects()) { + if (expr.IsVolatile()) { return false; } } } - return function.side_effects == FunctionSideEffects::HAS_SIDE_EFFECTS ? false : Expression::IsFoldable(); + return function.stability == FunctionStability::VOLATILE ? false : Expression::IsFoldable(); } string BoundFunctionExpression::ToString() const { diff --git a/src/duckdb/src/planner/expression/bound_window_expression.cpp b/src/duckdb/src/planner/expression/bound_window_expression.cpp index f34ab3619..7dd33ca82 100644 --- a/src/duckdb/src/planner/expression/bound_window_expression.cpp +++ b/src/duckdb/src/planner/expression/bound_window_expression.cpp @@ -12,7 +12,7 @@ BoundWindowExpression::BoundWindowExpression(ExpressionType type, LogicalType re unique_ptr aggregate, unique_ptr bind_info) : Expression(type, ExpressionClass::BOUND_WINDOW, std::move(return_type)), aggregate(std::move(aggregate)), - bind_info(std::move(bind_info)), ignore_nulls(false) { + bind_info(std::move(bind_info)), ignore_nulls(false), distinct(false) { } string BoundWindowExpression::ToString() const { @@ -30,6 +30,9 @@ bool BoundWindowExpression::Equals(const BaseExpression &other_p) const { if (ignore_nulls != other.ignore_nulls) { return false; } + if (distinct != other.distinct) { + return false; + } if (start != other.start || end != other.end) { return false; } @@ -43,6 +46,12 @@ bool BoundWindowExpression::Equals(const BaseExpression &other_p) const { return false; } } + // If there's function data, check if they are equal + if (bind_info.get() != other.bind_info.get()) { + if (!bind_info || !other.bind_info || !bind_info->Equals(*other.bind_info)) { + return false; + } + } // check if the child expressions are equivalent if (!Expression::ListEquals(children, other.children)) { return false; @@ -145,6 +154,7 @@ unique_ptr BoundWindowExpression::Copy() { new_window->offset_expr = offset_expr ? offset_expr->Copy() : nullptr; new_window->default_expr = default_expr ? default_expr->Copy() : nullptr; new_window->ignore_nulls = ignore_nulls; + new_window->distinct = distinct; for (auto &es : expr_stats) { if (es) { @@ -175,6 +185,7 @@ void BoundWindowExpression::Serialize(Serializer &serializer) const { 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); + serializer.WriteProperty(213, "distinct", distinct); } unique_ptr BoundWindowExpression::Deserialize(Deserializer &deserializer) { @@ -203,6 +214,7 @@ unique_ptr BoundWindowExpression::Deserialize(Deserializer &deserial 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); + deserializer.ReadProperty(213, "distinct", result->distinct); return std::move(result); } diff --git a/src/duckdb/src/planner/expression_binder.cpp b/src/duckdb/src/planner/expression_binder.cpp index f8bd2c9ca..fa03fabdf 100644 --- a/src/duckdb/src/planner/expression_binder.cpp +++ b/src/duckdb/src/planner/expression_binder.cpp @@ -1,5 +1,6 @@ #include "duckdb/planner/expression_binder.hpp" +#include "duckdb/catalog/catalog_entry/scalar_function_catalog_entry.hpp" #include "duckdb/parser/expression/list.hpp" #include "duckdb/parser/parsed_expression_iterator.hpp" #include "duckdb/planner/binder.hpp" @@ -71,11 +72,11 @@ BindResult ExpressionBinder::BindExpression(unique_ptr &expr, return BindExpression(expr_ref.Cast(), depth); case ExpressionClass::FUNCTION: { auto &function = expr_ref.Cast(); - if (function.function_name == "unnest" || function.function_name == "unlist") { + if (IsUnnestFunction(function.function_name)) { // special case, not in catalog return BindUnnest(function, depth, root_expression); } - // binding function expression has extra parameter needed for macro's + // binding a function expression requires an extra parameter for macros return BindExpression(function, depth, expr); } case ExpressionClass::LAMBDA: @@ -90,13 +91,13 @@ BindResult ExpressionBinder::BindExpression(unique_ptr &expr, return BindPositionalReference(expr, depth, root_expression); } case ExpressionClass::STAR: - return BindResult(binder.FormatError(expr_ref, "STAR expression is not supported here")); + return BindResult(BinderException(expr_ref, "STAR expression is not supported here")); default: throw NotImplementedException("Unimplemented expression class"); } } -BindResult ExpressionBinder::BindCorrelatedColumns(unique_ptr &expr, string error_message) { +BindResult ExpressionBinder::BindCorrelatedColumns(unique_ptr &expr, ErrorData error_message) { // try to bind in one of the outer queries, if the binding error occurred in a subquery auto &active_binders = binder.GetActiveBinders(); // make a copy of the set of binders, so we can restore it later @@ -109,7 +110,7 @@ BindResult ExpressionBinder::BindCorrelatedColumns(unique_ptr auto &next_binder = active_binders.back().get(); ExpressionBinder::QualifyColumnNames(next_binder.binder, expr); bind_error = next_binder.Bind(expr, depth); - if (bind_error.empty()) { + if (!bind_error.HasError()) { break; } depth++; @@ -119,11 +120,11 @@ BindResult ExpressionBinder::BindCorrelatedColumns(unique_ptr return BindResult(bind_error); } -void ExpressionBinder::BindChild(unique_ptr &expr, idx_t depth, string &error) { +void ExpressionBinder::BindChild(unique_ptr &expr, idx_t depth, ErrorData &error) { if (expr) { - string bind_error = Bind(expr, depth); - if (error.empty()) { - error = bind_error; + ErrorData bind_error = Bind(expr, depth); + if (!error.HasError()) { + error = std::move(bind_error); } } } @@ -216,14 +217,14 @@ unique_ptr ExpressionBinder::Bind(unique_ptr &expr bool root_expression) { // bind the main expression auto error_msg = Bind(expr, 0, root_expression); - if (!error_msg.empty()) { + if (error_msg.HasError()) { // Try binding the correlated column. If binding the correlated column // has error messages, those should be propagated up. So for the test case // having subquery failed to bind:14 the real error message should be something like // aggregate with constant input must be bound to a root node. auto result = BindCorrelatedColumns(expr, error_msg); if (result.HasError()) { - throw BinderException(result.error); + result.error.Throw(); } auto &bound_expr = expr->Cast(); ExtractCorrelatedExpressions(binder, *bound_expr.expr); @@ -252,27 +253,45 @@ unique_ptr ExpressionBinder::Bind(unique_ptr &expr return result; } -string ExpressionBinder::Bind(unique_ptr &expr, idx_t depth, bool root_expression) { +ErrorData ExpressionBinder::Bind(unique_ptr &expr, idx_t depth, bool root_expression) { // bind the node, but only if it has not been bound yet + auto query_location = expr->query_location; auto &expression = *expr; auto alias = expression.alias; if (expression.GetExpressionClass() == ExpressionClass::BOUND_EXPRESSION) { // already bound, don't bind it again - return string(); + return ErrorData(); } // bind the expression BindResult result = BindExpression(expr, depth, root_expression); if (result.HasError()) { - return result.error; + return std::move(result.error); } // successfully bound: replace the node with a BoundExpression + result.expression->query_location = query_location; expr = make_uniq(std::move(result.expression)); auto &be = expr->Cast(); be.alias = alias; if (!alias.empty()) { be.expr->alias = alias; } - return string(); + return ErrorData(); +} + +bool ExpressionBinder::IsUnnestFunction(const string &function_name) { + return function_name == "unnest" || function_name == "unlist"; +} + +bool ExpressionBinder::IsLambdaFunction(const FunctionExpression &function) { + // check for lambda parameters, ignore ->> operator (JSON extension) + if (function.function_name != "->>") { + for (auto &child : function.children) { + if (child->expression_class == ExpressionClass::LAMBDA) { + return true; + } + } + } + return false; } } // namespace duckdb diff --git a/src/duckdb/src/planner/expression_binder/base_select_binder.cpp b/src/duckdb/src/planner/expression_binder/base_select_binder.cpp index 57cdf6868..b4708561e 100644 --- a/src/duckdb/src/planner/expression_binder/base_select_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/base_select_binder.cpp @@ -88,7 +88,7 @@ BindResult BaseSelectBinder::BindColumnRef(unique_ptr &expr_pt "cannot be referenced before it is defined", colref.column_names[0]); } - if (node.select_list[index]->HasSideEffects()) { + if (node.select_list[index]->IsVolatile()) { throw BinderException("Alias \"%s\" referenced in a SELECT clause - but the expression has side " "effects. This is not yet supported.", colref.column_names[0]); @@ -112,10 +112,10 @@ BindResult BaseSelectBinder::BindGroupingFunction(OperatorExpression &op, idx_t throw InternalException("GROUPING requires at least one child"); } if (node.groups.group_expressions.empty()) { - return BindResult(binder.FormatError(op, "GROUPING statement cannot be used without groups")); + return BindResult(BinderException(op, "GROUPING statement cannot be used without groups")); } if (op.children.size() >= 64) { - return BindResult(binder.FormatError(op, "GROUPING statement cannot have more than 64 groups")); + return BindResult(BinderException(op, "GROUPING statement cannot have more than 64 groups")); } vector group_indexes; group_indexes.reserve(op.children.size()); @@ -123,8 +123,7 @@ BindResult BaseSelectBinder::BindGroupingFunction(OperatorExpression &op, idx_t ExpressionBinder::QualifyColumnNames(binder, child); auto idx = TryBindGroup(*child, depth); if (idx == DConstants::INVALID_INDEX) { - return BindResult(binder.FormatError( - op, StringUtil::Format("GROUPING child \"%s\" must be a grouping column", child->GetName()))); + return BindResult(BinderException(op, "GROUPING child \"%s\" must be a grouping column", child->GetName())); } group_indexes.push_back(idx); } @@ -150,7 +149,7 @@ BindResult BaseSelectBinder::BindGroup(ParsedExpression &expr, idx_t depth, idx_ bool BaseSelectBinder::QualifyColumnAlias(const ColumnRefExpression &colref) { if (!colref.IsQualified()) { - return alias_map.find(colref.column_names[0]) != alias_map.end() ? true : false; + return alias_map.find(colref.column_names[0]) != alias_map.end(); } return false; } diff --git a/src/duckdb/src/planner/expression_binder/check_binder.cpp b/src/duckdb/src/planner/expression_binder/check_binder.cpp index 698d9e065..c89c96ded 100644 --- a/src/duckdb/src/planner/expression_binder/check_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/check_binder.cpp @@ -44,14 +44,15 @@ BindResult ExpressionBinder::BindQualifiedColumnName(ColumnRefExpression &colref BindResult CheckBinder::BindCheckColumn(ColumnRefExpression &colref) { - // if this is a lambda parameters, then we temporarily add a BoundLambdaRef, - // which we capture and remove later - if (lambda_bindings) { - for (idx_t i = 0; i < lambda_bindings->size(); i++) { - if (colref.GetColumnName() == (*lambda_bindings)[i].dummy_name) { - // FIXME: support lambdas in CHECK constraints - // FIXME: like so: return (*lambda_bindings)[i].Bind(colref, i, depth); - throw NotImplementedException("Lambda functions are currently not supported in CHECK constraints."); + if (!colref.IsQualified()) { + if (lambda_bindings) { + for (idx_t i = lambda_bindings->size(); i > 0; i--) { + if ((*lambda_bindings)[i - 1].HasMatchingBinding(colref.GetName())) { + // FIXME: support lambdas in CHECK constraints + // FIXME: like so: return (*lambda_bindings)[i - 1].Bind(colref, i, depth); + // FIXME: and move this to LambdaRefExpression::FindMatchingBinding + throw NotImplementedException("Lambda functions are currently not supported in CHECK constraints."); + } } } } diff --git a/src/duckdb/src/planner/expression_binder/column_alias_binder.cpp b/src/duckdb/src/planner/expression_binder/column_alias_binder.cpp index 7dace55c5..6496c919a 100644 --- a/src/duckdb/src/planner/expression_binder/column_alias_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/column_alias_binder.cpp @@ -12,19 +12,22 @@ ColumnAliasBinder::ColumnAliasBinder(BoundSelectNode &node, const case_insensiti : node(node), alias_map(alias_map), visited_select_indexes() { } -BindResult ColumnAliasBinder::BindAlias(ExpressionBinder &enclosing_binder, ColumnRefExpression &expr, idx_t depth, - bool root_expression) { +bool ColumnAliasBinder::BindAlias(ExpressionBinder &enclosing_binder, ColumnRefExpression &expr, idx_t depth, + bool root_expression, BindResult &result) { if (expr.IsQualified()) { - return BindResult(StringUtil::Format("Alias %s cannot be qualified.", expr.ToString())); + // qualified columns cannot be aliases + return false; } auto alias_entry = alias_map.find(expr.column_names[0]); if (alias_entry == alias_map.end()) { - return BindResult(StringUtil::Format("Alias %s is not found.", expr.ToString())); + // no alias found + return false; } if (visited_select_indexes.find(alias_entry->second) != visited_select_indexes.end()) { - return BindResult("Cannot resolve self-referential alias"); + // self-referential alias cannot be resolved + return false; } // found an alias: bind the alias expression @@ -33,9 +36,9 @@ BindResult ColumnAliasBinder::BindAlias(ExpressionBinder &enclosing_binder, Colu // since the alias has been found, pass a depth of 0. See Issue 4978 (#16) // ColumnAliasBinders are only in Having, Qualify and Where Binders - auto result = enclosing_binder.BindExpression(expression, depth, root_expression); + result = enclosing_binder.BindExpression(expression, depth, root_expression); visited_select_indexes.erase(alias_entry->second); - return result; + return true; } } // namespace duckdb diff --git a/src/duckdb/src/planner/expression_binder/having_binder.cpp b/src/duckdb/src/planner/expression_binder/having_binder.cpp index a314c579d..4df16339e 100644 --- a/src/duckdb/src/planner/expression_binder/having_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/having_binder.cpp @@ -17,17 +17,21 @@ HavingBinder::HavingBinder(Binder &binder, ClientContext &context, BoundSelectNo BindResult HavingBinder::BindColumnRef(unique_ptr &expr_ptr, idx_t depth, bool root_expression) { auto &expr = expr_ptr->Cast(); - auto alias_result = column_alias_binder.BindAlias(*this, expr, depth, root_expression); - if (!alias_result.HasError()) { + BindResult alias_result; + auto found_alias = column_alias_binder.BindAlias(*this, expr, depth, root_expression, alias_result); + if (found_alias) { if (depth > 0) { - throw BinderException("Having clause cannot reference alias in correlated subquery"); + throw BinderException("Having clause cannot reference alias \"%s\" in correlated subquery", + expr.GetColumnName()); } return alias_result; } if (aggregate_handling == AggregateHandling::FORCE_AGGREGATES) { if (depth > 0) { - throw BinderException("Having clause cannot reference column in correlated subquery and group by all"); + throw BinderException( + "Having clause cannot reference column \"%s\" in correlated subquery and group by all", + expr.GetColumnName()); } auto expr = duckdb::BaseSelectBinder::BindExpression(expr_ptr, depth); if (expr.HasError()) { diff --git a/src/duckdb/src/planner/expression_binder/qualify_binder.cpp b/src/duckdb/src/planner/expression_binder/qualify_binder.cpp index 59a97e617..ceaf916a9 100644 --- a/src/duckdb/src/planner/expression_binder/qualify_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/qualify_binder.cpp @@ -22,8 +22,9 @@ BindResult QualifyBinder::BindColumnRef(unique_ptr &expr_ptr, return result; } - auto alias_result = column_alias_binder.BindAlias(*this, expr, depth, root_expression); - if (!alias_result.HasError()) { + BindResult alias_result; + auto found_alias = column_alias_binder.BindAlias(*this, expr, depth, root_expression, alias_result); + if (found_alias) { return alias_result; } 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 f6f749585..98a73ed36 100644 --- a/src/duckdb/src/planner/expression_binder/table_function_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/table_function_binder.cpp @@ -10,21 +10,18 @@ TableFunctionBinder::TableFunctionBinder(Binder &binder, ClientContext &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); + auto &lambda_ref = expr.Cast(); + return (*lambda_bindings)[expr.lambda_idx].Bind(lambda_ref, depth); } BindResult TableFunctionBinder::BindColumnReference(ColumnRefExpression &expr, idx_t depth, bool root_expression) { - // if this is a lambda parameters, then we temporarily add a BoundLambdaRef, - // which we capture and remove later - if (lambda_bindings) { - auto &colref = expr.Cast(); - for (idx_t i = 0; i < lambda_bindings->size(); i++) { - if ((*lambda_bindings)[i].HasMatchingBinding(colref.GetColumnName())) { - auto lambdaref = make_uniq(i, colref.GetColumnName()); - return BindLambdaReference(*lambdaref, depth); - } + // try binding as a lambda parameter + auto &col_ref = expr.Cast(); + if (!col_ref.IsQualified()) { + auto lambda_ref = LambdaRefExpression::FindMatchingBinding(lambda_bindings, col_ref.GetName()); + if (lambda_ref) { + return BindLambdaReference(lambda_ref->Cast(), depth); } } diff --git a/src/duckdb/src/planner/expression_binder/where_binder.cpp b/src/duckdb/src/planner/expression_binder/where_binder.cpp index 182687f83..9c868f671 100644 --- a/src/duckdb/src/planner/expression_binder/where_binder.cpp +++ b/src/duckdb/src/planner/expression_binder/where_binder.cpp @@ -16,12 +16,11 @@ BindResult WhereBinder::BindColumnRef(unique_ptr &expr_ptr, id return result; } - BindResult alias_result = column_alias_binder->BindAlias(*this, expr, depth, root_expression); - // This code path cannot be exercised at thispoint. #1547 might change that. - if (!alias_result.HasError()) { + BindResult alias_result; + auto found_alias = column_alias_binder->BindAlias(*this, expr, depth, root_expression, alias_result); + if (found_alias) { return alias_result; } - return result; } diff --git a/src/duckdb/src/planner/expression_iterator.cpp b/src/duckdb/src/planner/expression_iterator.cpp index 34e6bc108..5b2ffbe87 100644 --- a/src/duckdb/src/planner/expression_iterator.cpp +++ b/src/duckdb/src/planner/expression_iterator.cpp @@ -175,7 +175,7 @@ void ExpressionIterator::EnumerateTableRefChildren(BoundTableRef &ref, break; } case TableReferenceType::TABLE_FUNCTION: - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: case TableReferenceType::BASE_TABLE: case TableReferenceType::CTE: break; diff --git a/src/duckdb/src/planner/filter/constant_filter.cpp b/src/duckdb/src/planner/filter/constant_filter.cpp index 3f5084d5d..1388aa42a 100644 --- a/src/duckdb/src/planner/filter/constant_filter.cpp +++ b/src/duckdb/src/planner/filter/constant_filter.cpp @@ -15,6 +15,7 @@ FilterPropagateResult ConstantFilter::CheckStatistics(BaseStatistics &stats) { case PhysicalType::UINT16: case PhysicalType::UINT32: case PhysicalType::UINT64: + case PhysicalType::UINT128: case PhysicalType::INT8: case PhysicalType::INT16: case PhysicalType::INT32: diff --git a/src/duckdb/src/planner/filter/struct_filter.cpp b/src/duckdb/src/planner/filter/struct_filter.cpp new file mode 100644 index 000000000..0148f95e7 --- /dev/null +++ b/src/duckdb/src/planner/filter/struct_filter.cpp @@ -0,0 +1,33 @@ +#include "duckdb/planner/filter/struct_filter.hpp" +#include "duckdb/storage/statistics/base_statistics.hpp" +#include "duckdb/storage/statistics/struct_stats.hpp" +#include "duckdb/common/string_util.hpp" + +namespace duckdb { + +StructFilter::StructFilter(idx_t child_idx_p, string child_name_p, unique_ptr child_filter_p) + : TableFilter(TableFilterType::STRUCT_EXTRACT), child_idx(child_idx_p), child_name(std::move(child_name_p)), + child_filter(std::move(child_filter_p)) { +} + +FilterPropagateResult StructFilter::CheckStatistics(BaseStatistics &stats) { + D_ASSERT(stats.GetType().id() == LogicalTypeId::STRUCT); + // Check the child statistics + auto &child_stats = StructStats::GetChildStats(stats, child_idx); + return child_filter->CheckStatistics(child_stats); +} + +string StructFilter::ToString(const string &column_name) { + return child_filter->ToString(column_name + "." + child_name); +} + +bool StructFilter::Equals(const TableFilter &other_p) const { + if (!TableFilter::Equals(other_p)) { + return false; + } + auto &other = other_p.Cast(); + return other.child_idx == child_idx && StringUtil::CIEquals(other.child_name, child_name) && + other.child_filter->Equals(*child_filter); +} + +} // namespace duckdb diff --git a/src/duckdb/src/planner/joinside.cpp b/src/duckdb/src/planner/joinside.cpp index 3b24f337b..ba5b49cc3 100644 --- a/src/duckdb/src/planner/joinside.cpp +++ b/src/duckdb/src/planner/joinside.cpp @@ -60,7 +60,7 @@ JoinSide JoinSide::GetJoinSide(Expression &expression, const unordered_set(); if (colref.depth > 0) { - throw Exception("Non-inner join on correlated columns not supported"); + throw NotImplementedException("Non-inner join on correlated columns not supported"); } return GetJoinSide(colref.binding.table_index, left_bindings, right_bindings); } diff --git a/src/duckdb/src/planner/logical_operator.cpp b/src/duckdb/src/planner/logical_operator.cpp index 8c3c4c93b..b7b9dc9f3 100644 --- a/src/duckdb/src/planner/logical_operator.cpp +++ b/src/duckdb/src/planner/logical_operator.cpp @@ -185,9 +185,10 @@ unique_ptr LogicalOperator::Copy(ClientContext &context) const this->Serialize(serializer); serializer.End(); } catch (NotImplementedException &ex) { + ErrorData error(ex); throw NotImplementedException("Logical Operator Copy requires the logical operator and all of its children to " "be serializable: " + - std::string(ex.what())); + error.RawMessage()); } stream.Rewind(); bound_parameter_map_t parameters; diff --git a/src/duckdb/src/planner/operator/logical_copy_to_file.cpp b/src/duckdb/src/planner/operator/logical_copy_to_file.cpp index 93572634c..67f684d84 100644 --- a/src/duckdb/src/planner/operator/logical_copy_to_file.cpp +++ b/src/duckdb/src/planner/operator/logical_copy_to_file.cpp @@ -1,12 +1,11 @@ #include "duckdb/planner/operator/logical_copy_to_file.hpp" #include "duckdb/catalog/catalog_entry/copy_function_catalog_entry.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/common/serializer/serializer.hpp" #include "duckdb/function/copy_function.hpp" #include "duckdb/function/function_serialization.hpp" -#include "duckdb/common/serializer/serializer.hpp" -#include "duckdb/common/serializer/deserializer.hpp" - namespace duckdb { void LogicalCopyToFile::Serialize(Serializer &serializer) const { @@ -32,6 +31,8 @@ void LogicalCopyToFile::Serialize(Serializer &serializer) const { serializer.WriteObject(212, "function_data", [&](Serializer &obj) { function.serialize(obj, *bind_data, function); }); } + + serializer.WriteProperty(213, "file_extension", file_extension); } unique_ptr LogicalCopyToFile::Deserialize(Deserializer &deserializer) { @@ -70,13 +71,21 @@ unique_ptr LogicalCopyToFile::Deserialize(Deserializer &deseria if (!function.copy_to_bind) { throw InternalException("Copy function \"%s\" has neither bind nor (de)serialize", function.name); } - bind_data = function.copy_to_bind(context, *copy_info, names, expected_types); + + CopyFunctionBindInput function_bind_input(*copy_info); + bind_data = function.copy_to_bind(context, function_bind_input, names, expected_types); } + auto default_extension = function.extension; + + auto file_extension = + deserializer.ReadPropertyWithDefault(213, "file_extension", std::move(default_extension)); + auto result = make_uniq(function, std::move(bind_data), std::move(copy_info)); result->file_path = file_path; result->use_tmp_file = use_tmp_file; result->filename_pattern = filename_pattern; + result->file_extension = file_extension; result->overwrite_or_ignore = overwrite_or_ignore; result->per_thread_output = per_thread_output; result->partition_output = partition_output; diff --git a/src/duckdb/src/planner/planner.cpp b/src/duckdb/src/planner/planner.cpp index 4b1c28d6a..7194c10d3 100644 --- a/src/duckdb/src/planner/planner.cpp +++ b/src/duckdb/src/planner/planner.cpp @@ -47,32 +47,33 @@ void Planner::CreatePlan(SQLStatement &statement) { auto max_tree_depth = ClientConfig::GetConfig(context).max_expression_depth; CheckTreeDepth(*plan, max_tree_depth); - } catch (const ParameterNotResolvedException &ex) { - // parameter types could not be resolved - this->names = {"unknown"}; - this->types = {LogicalTypeId::UNKNOWN}; + } catch (const std::exception &ex) { + ErrorData error(ex); this->plan = nullptr; - parameters_resolved = false; - } catch (const Exception &ex) { - auto &config = DBConfig::GetConfig(context); - - this->plan = nullptr; - for (auto &extension_op : config.operator_extensions) { - auto bound_statement = - extension_op->Bind(context, *this->binder, extension_op->operator_info.get(), statement); - if (bound_statement.plan != nullptr) { - this->names = bound_statement.names; - this->types = bound_statement.types; - this->plan = std::move(bound_statement.plan); - break; + if (error.Type() == ExceptionType::PARAMETER_NOT_RESOLVED) { + // parameter types could not be resolved + this->names = {"unknown"}; + this->types = {LogicalTypeId::UNKNOWN}; + parameters_resolved = false; + } else if (error.Type() != ExceptionType::INVALID) { + // different exception type - try operator_extensions + auto &config = DBConfig::GetConfig(context); + for (auto &extension_op : config.operator_extensions) { + auto bound_statement = + extension_op->Bind(context, *this->binder, extension_op->operator_info.get(), statement); + if (bound_statement.plan != nullptr) { + this->names = bound_statement.names; + this->types = bound_statement.types; + this->plan = std::move(bound_statement.plan); + break; + } } - } - - if (!this->plan) { + if (!this->plan) { + throw; + } + } else { throw; } - } catch (std::exception &ex) { - throw; } this->properties = binder->properties; this->properties.parameter_count = parameter_count; @@ -127,7 +128,6 @@ void Planner::CreatePlan(unique_ptr statement) { case StatementType::CALL_STATEMENT: case StatementType::EXPORT_STATEMENT: case StatementType::PRAGMA_STATEMENT: - case StatementType::SHOW_STATEMENT: case StatementType::SET_STATEMENT: case StatementType::LOAD_STATEMENT: case StatementType::EXTENSION_STATEMENT: diff --git a/src/duckdb/src/planner/pragma_handler.cpp b/src/duckdb/src/planner/pragma_handler.cpp index f0a2ec221..7e0b8eab7 100644 --- a/src/duckdb/src/planner/pragma_handler.cpp +++ b/src/duckdb/src/planner/pragma_handler.cpp @@ -70,7 +70,7 @@ void PragmaHandler::HandlePragmaStatements(ClientContextLock &lock, vector().info->Copy(); - QueryErrorContext error_context(&statement, statement.stmt_location); + QueryErrorContext error_context(statement.stmt_location); auto binder = Binder::CreateBinder(context); auto bound_info = binder->BindPragma(*info, error_context); if (bound_info->function.query) { diff --git a/src/duckdb/src/planner/subquery/flatten_dependent_join.cpp b/src/duckdb/src/planner/subquery/flatten_dependent_join.cpp index 9c3632fec..81fa3e963 100644 --- a/src/duckdb/src/planner/subquery/flatten_dependent_join.cpp +++ b/src/duckdb/src/planner/subquery/flatten_dependent_join.cpp @@ -284,7 +284,7 @@ unique_ptr FlattenDependentJoins::PushDownDependentJoinInternal case LogicalOperatorType::LOGICAL_DEPENDENT_JOIN: { auto &dependent_join = plan->Cast(); if (!((dependent_join.join_type == JoinType::INNER) || (dependent_join.join_type == JoinType::LEFT))) { - throw Exception("Dependent join can only be INNER or LEFT type"); + throw NotImplementedException("Dependent join can only be INNER or LEFT type"); } D_ASSERT(plan->children.size() == 2); // Push all the bindings down to the left side so the right side knows where to refer DELIM_GET from @@ -345,7 +345,7 @@ unique_ptr FlattenDependentJoins::PushDownDependentJoinInternal } } else if (join.join_type == JoinType::MARK) { if (right_has_correlation) { - throw Exception("MARK join with correlation in RHS not supported"); + throw NotImplementedException("MARK join with correlation in RHS not supported"); } // push the child into the LHS plan->children[0] = PushDownDependentJoinInternal(std::move(plan->children[0]), @@ -355,7 +355,7 @@ unique_ptr FlattenDependentJoins::PushDownDependentJoinInternal rewriter.VisitOperator(*plan); return plan; } else { - throw Exception("Unsupported join type for flattening correlated subquery"); + throw NotImplementedException("Unsupported join type for flattening correlated subquery"); } // both sides have correlation // push into both sides diff --git a/src/duckdb/src/planner/table_binding.cpp b/src/duckdb/src/planner/table_binding.cpp index 3f1d572e7..1709a4524 100644 --- a/src/duckdb/src/planner/table_binding.cpp +++ b/src/duckdb/src/planner/table_binding.cpp @@ -52,8 +52,9 @@ bool Binding::HasMatchingBinding(const string &column_name) { return TryGetBindingIndex(column_name, result); } -string Binding::ColumnNotFoundError(const string &column_name) const { - return StringUtil::Format("Values list \"%s\" does not have a column named \"%s\"", alias, column_name); +ErrorData Binding::ColumnNotFoundError(const string &column_name) const { + return ErrorData(ExceptionType::BINDER, + StringUtil::Format("Values list \"%s\" does not have a column named \"%s\"", alias, column_name)); } BindResult Binding::Bind(ColumnRefExpression &colref, idx_t depth) { @@ -218,8 +219,9 @@ optional_ptr TableBinding::GetStandardEntry() { return entry; } -string TableBinding::ColumnNotFoundError(const string &column_name) const { - return StringUtil::Format("Table \"%s\" does not have a column named \"%s\"", alias, column_name); +ErrorData TableBinding::ColumnNotFoundError(const string &column_name) const { + return ErrorData(ExceptionType::BINDER, + StringUtil::Format("Table \"%s\" does not have a column named \"%s\"", alias, column_name)); } DummyBinding::DummyBinding(vector types, vector names, string dummy_name) diff --git a/src/duckdb/src/storage/buffer/buffer_pool.cpp b/src/duckdb/src/storage/buffer/buffer_pool.cpp index 482296734..77d1bf8a3 100644 --- a/src/duckdb/src/storage/buffer/buffer_pool.cpp +++ b/src/duckdb/src/storage/buffer/buffer_pool.cpp @@ -1,6 +1,8 @@ #include "duckdb/storage/buffer/buffer_pool.hpp" -#include "duckdb/parallel/concurrentqueue.hpp" + #include "duckdb/common/exception.hpp" +#include "duckdb/parallel/concurrentqueue.hpp" +#include "duckdb/storage/temporary_memory_manager.hpp" namespace duckdb { @@ -33,7 +35,8 @@ shared_ptr BufferEvictionNode::TryGetBlockHandle() { } BufferPool::BufferPool(idx_t maximum_memory) - : current_memory(0), maximum_memory(maximum_memory), queue(make_uniq()), queue_insertions(0) { + : current_memory(0), maximum_memory(maximum_memory), queue(make_uniq()), queue_insertions(0), + temporary_memory_manager(make_uniq()) { } BufferPool::~BufferPool() { } @@ -57,10 +60,19 @@ void BufferPool::IncreaseUsedMemory(idx_t size) { idx_t BufferPool::GetUsedMemory() const { return current_memory; } + idx_t BufferPool::GetMaxMemory() const { return maximum_memory; } +idx_t BufferPool::GetQueryMaxMemory() const { + return GetMaxMemory(); +} + +TemporaryMemoryManager &BufferPool::GetTemporaryMemoryManager() { + return *temporary_memory_manager; +} + BufferPool::EvictionResult BufferPool::EvictBlocks(idx_t extra_memory, idx_t memory_limit, unique_ptr *buffer) { BufferEvictionNode node; diff --git a/src/duckdb/src/storage/buffer_manager.cpp b/src/duckdb/src/storage/buffer_manager.cpp index 9cd61aecf..c9b880860 100644 --- a/src/duckdb/src/storage/buffer_manager.cpp +++ b/src/duckdb/src/storage/buffer_manager.cpp @@ -1,7 +1,9 @@ #include "duckdb/storage/buffer_manager.hpp" + #include "duckdb/common/allocator.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/file_buffer.hpp" +#include "duckdb/storage/buffer/buffer_pool.hpp" #include "duckdb/storage/standard_buffer_manager.hpp" namespace duckdb { @@ -37,10 +39,14 @@ const string &BufferManager::GetTemporaryDirectory() { throw InternalException("This type of BufferManager does not allow a temporary directory"); } -BufferPool &BufferManager::GetBufferPool() { +BufferPool &BufferManager::GetBufferPool() const { throw InternalException("This type of BufferManager does not have a buffer pool"); } +TemporaryMemoryManager &BufferManager::GetTemporaryMemoryManager() { + throw NotImplementedException("This type of BufferManager does not have a TemporaryMemoryManager"); +} + void BufferManager::SetTemporaryDirectory(const string &new_dir) { throw NotImplementedException("This type of BufferManager can not set a temporary directory"); } @@ -53,6 +59,11 @@ bool BufferManager::HasTemporaryDirectory() const { return false; } +//! Returns the maximum available memory for a given query +idx_t BufferManager::GetQueryMaxMemory() const { + return GetBufferPool().GetQueryMaxMemory(); +} + unique_ptr BufferManager::ConstructManagedBuffer(idx_t size, unique_ptr &&source, FileBufferType type) { throw NotImplementedException("This type of BufferManager can not construct managed buffers"); diff --git a/src/duckdb/src/storage/checkpoint_manager.cpp b/src/duckdb/src/storage/checkpoint_manager.cpp index a98ad69c4..1f97b538f 100644 --- a/src/duckdb/src/storage/checkpoint_manager.cpp +++ b/src/duckdb/src/storage/checkpoint_manager.cpp @@ -7,10 +7,10 @@ #include "duckdb/catalog/catalog_entry/sequence_catalog_entry.hpp" #include "duckdb/catalog/catalog_entry/type_catalog_entry.hpp" #include "duckdb/catalog/catalog_entry/view_catalog_entry.hpp" +#include "duckdb/catalog/catalog_entry/index_catalog_entry.hpp" #include "duckdb/catalog/duck_catalog.hpp" #include "duckdb/common/serializer/binary_deserializer.hpp" #include "duckdb/common/serializer/binary_serializer.hpp" -#include "duckdb/execution/index/art/art.hpp" #include "duckdb/main/attached_database.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/config.hpp" @@ -28,6 +28,8 @@ #include "duckdb/storage/metadata/metadata_reader.hpp" #include "duckdb/storage/table/column_checkpoint_state.hpp" #include "duckdb/transaction/transaction_manager.hpp" +#include "duckdb/execution/index/art/art.hpp" +#include "duckdb/execution/index/unknown_index.hpp" namespace duckdb { @@ -184,8 +186,11 @@ void SingleFileCheckpointWriter::CreateCheckpoint() { // CHECKPOINT "meta_block_id", and the id MATCHES the head idin the file we know that the database was successfully // checkpointed, so we know that we should avoid replaying the WAL to avoid duplicating data auto wal = storage_manager.GetWriteAheadLog(); - wal->WriteCheckpoint(meta_block); - wal->Flush(); + bool wal_is_empty = wal->GetWALSize() == 0; + if (!wal_is_empty) { + wal->WriteCheckpoint(meta_block); + wal->Flush(); + } if (config.options.checkpoint_abort == CheckpointAbort::DEBUG_ABORT_BEFORE_HEADER) { throw FatalException("Checkpoint aborted before header write because of PRAGMA checkpoint_abort flag"); @@ -194,6 +199,8 @@ void SingleFileCheckpointWriter::CreateCheckpoint() { // finally write the updated header DatabaseHeader header; header.meta_block = meta_block.block_pointer; + header.block_size = Storage::BLOCK_ALLOC_SIZE; + header.vector_size = STANDARD_VECTOR_SIZE; block_manager.WriteHeader(header); if (config.options.checkpoint_abort == CheckpointAbort::DEBUG_ABORT_BEFORE_TRUNCATE) { @@ -204,7 +211,9 @@ void SingleFileCheckpointWriter::CreateCheckpoint() { block_manager.Truncate(); // truncate the WAL - wal->Truncate(0); + if (!wal_is_empty) { + wal->Truncate(0); + } } void CheckpointReader::LoadCheckpoint(ClientContext &context, MetadataReader &reader) { @@ -395,10 +404,21 @@ void CheckpointReader::ReadIndex(ClientContext &context, Deserializer &deseriali deserializer.ReadPropertyWithDefault(101, "root_block_pointer", BlockPointer()); // create the index in the catalog + + // look for the table in the catalog auto &table = catalog.GetEntry(context, CatalogType::TABLE_ENTRY, create_info->schema, info.table).Cast(); + + // we also need to make sure the index type is loaded + // backwards compatability: + // if the index type is not specified, we default to ART + if (info.index_type.empty()) { + info.index_type = ART::TYPE_NAME; + } + + // now we can look for the index in the catalog and assign the table info auto &index = catalog.CreateIndex(context, info)->Cast(); - index.info = table.GetStorage().info; + index.info = make_shared(table.GetStorage().info, info.index_name); // insert the parsed expressions into the index so that we can (de)serialize them during consecutive checkpoints for (auto &parsed_expr : info.parsed_expressions) { @@ -451,9 +471,20 @@ void CheckpointReader::ReadIndex(ClientContext &context, Deserializer &deseriali } D_ASSERT(index_storage_info.IsValid() && !index_storage_info.name.empty()); - auto art = make_uniq(info.index_name, info.constraint_type, info.column_ids, TableIOManager::Get(data_table), - std::move(unbound_expressions), data_table.db, nullptr, index_storage_info); - data_table.info->indexes.AddIndex(std::move(art)); + + // This is executed before any extensions can be loaded, which is why we must treat any index type that is not + // built-in (ART) as unknown + if (info.index_type == ART::TYPE_NAME) { + data_table.info->indexes.AddIndex(make_uniq(info.index_name, info.constraint_type, info.column_ids, + TableIOManager::Get(data_table), unbound_expressions, + data_table.db, nullptr, index_storage_info)); + } else { + auto unknown_index = make_uniq(info.index_name, info.index_type, info.constraint_type, + info.column_ids, TableIOManager::Get(data_table), + unbound_expressions, data_table.db, info, index_storage_info); + + data_table.info->indexes.AddIndex(std::move(unknown_index)); + } } //===--------------------------------------------------------------------===// diff --git a/src/duckdb/src/storage/compression/alp/alp.cpp b/src/duckdb/src/storage/compression/alp/alp.cpp new file mode 100644 index 000000000..62d9caa22 --- /dev/null +++ b/src/duckdb/src/storage/compression/alp/alp.cpp @@ -0,0 +1,57 @@ +#include "duckdb/storage/compression/alp/alp_compress.hpp" +#include "duckdb/storage/compression/alp/alp_scan.hpp" +#include "duckdb/storage/compression/alp/alp_fetch.hpp" +#include "duckdb/storage/compression/alp/alp_analyze.hpp" + +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" +#include "duckdb/main/config.hpp" + +#include +#include + +namespace duckdb { + +template +CompressionFunction GetAlpFunction(PhysicalType data_type) { + throw NotImplementedException("GetAlpFunction not implemented for the given datatype"); +} + +template <> +CompressionFunction GetAlpFunction(PhysicalType data_type) { + return CompressionFunction(CompressionType::COMPRESSION_ALP, data_type, AlpInitAnalyze, AlpAnalyze, + AlpFinalAnalyze, AlpInitCompression, AlpCompress, + AlpFinalizeCompress, AlpInitScan, AlpScan, AlpScanPartial, + AlpFetchRow, AlpSkip); +} + +template <> +CompressionFunction GetAlpFunction(PhysicalType data_type) { + return CompressionFunction(CompressionType::COMPRESSION_ALP, data_type, AlpInitAnalyze, AlpAnalyze, + AlpFinalAnalyze, AlpInitCompression, AlpCompress, + AlpFinalizeCompress, AlpInitScan, AlpScan, + AlpScanPartial, AlpFetchRow, AlpSkip); +} + +CompressionFunction AlpCompressionFun::GetFunction(PhysicalType type) { + switch (type) { + case PhysicalType::FLOAT: + return GetAlpFunction(type); + case PhysicalType::DOUBLE: + return GetAlpFunction(type); + default: + throw InternalException("Unsupported type for Alp"); + } +} + +bool AlpCompressionFun::TypeIsSupported(PhysicalType type) { + switch (type) { + case PhysicalType::FLOAT: + case PhysicalType::DOUBLE: + return true; + default: + return false; + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/compression/alp/alp_constants.cpp b/src/duckdb/src/storage/compression/alp/alp_constants.cpp new file mode 100644 index 000000000..694b5e61a --- /dev/null +++ b/src/duckdb/src/storage/compression/alp/alp_constants.cpp @@ -0,0 +1,13 @@ +#include "duckdb/storage/compression/alp/alp_constants.hpp" + +namespace duckdb { + +constexpr int64_t AlpConstants::FACT_ARR[]; + +constexpr float AlpTypedConstants::EXP_ARR[]; +constexpr float AlpTypedConstants::FRAC_ARR[]; + +constexpr double AlpTypedConstants::EXP_ARR[]; +constexpr double AlpTypedConstants::FRAC_ARR[]; + +} // namespace duckdb diff --git a/src/duckdb/src/storage/compression/alprd.cpp b/src/duckdb/src/storage/compression/alprd.cpp new file mode 100644 index 000000000..1ca623673 --- /dev/null +++ b/src/duckdb/src/storage/compression/alprd.cpp @@ -0,0 +1,57 @@ +#include "duckdb/storage/compression/alprd/alprd_analyze.hpp" +#include "duckdb/storage/compression/alprd/alprd_compress.hpp" +#include "duckdb/storage/compression/alprd/alprd_fetch.hpp" +#include "duckdb/storage/compression/alprd/alprd_scan.hpp" + +#include "duckdb/common/limits.hpp" +#include "duckdb/function/compression/compression.hpp" +#include "duckdb/function/compression_function.hpp" + +#include +#include + +namespace duckdb { + +template +CompressionFunction GetAlpRDFunction(PhysicalType data_type) { + throw NotImplementedException("GetAlpFunction not implemented for the given datatype"); +} + +template <> +CompressionFunction GetAlpRDFunction(PhysicalType data_type) { + return CompressionFunction(CompressionType::COMPRESSION_ALPRD, data_type, AlpRDInitAnalyze, + AlpRDAnalyze, AlpRDFinalAnalyze, AlpRDInitCompression, + AlpRDCompress, AlpRDFinalizeCompress, AlpRDInitScan, + AlpRDScan, AlpRDScanPartial, AlpRDFetchRow, AlpRDSkip); +} + +template <> +CompressionFunction GetAlpRDFunction(PhysicalType data_type) { + return CompressionFunction(CompressionType::COMPRESSION_ALPRD, data_type, AlpRDInitAnalyze, + AlpRDAnalyze, AlpRDFinalAnalyze, AlpRDInitCompression, + AlpRDCompress, AlpRDFinalizeCompress, AlpRDInitScan, + AlpRDScan, AlpRDScanPartial, AlpRDFetchRow, AlpRDSkip); +} + +CompressionFunction AlpRDCompressionFun::GetFunction(PhysicalType type) { + switch (type) { + case PhysicalType::FLOAT: + return GetAlpRDFunction(type); + case PhysicalType::DOUBLE: + return GetAlpRDFunction(type); + default: + throw InternalException("Unsupported type for Alp"); + } +} + +bool AlpRDCompressionFun::TypeIsSupported(PhysicalType type) { + switch (type) { + case PhysicalType::FLOAT: + case PhysicalType::DOUBLE: + return true; + default: + return false; + } +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/compression/bitpacking.cpp b/src/duckdb/src/storage/compression/bitpacking.cpp index f07ee91aa..fa7f4a7c7 100644 --- a/src/duckdb/src/storage/compression/bitpacking.cpp +++ b/src/duckdb/src/storage/compression/bitpacking.cpp @@ -1,19 +1,18 @@ #include "duckdb/common/bitpacking.hpp" #include "duckdb/common/limits.hpp" +#include "duckdb/common/numeric_utils.hpp" +#include "duckdb/common/operator/add.hpp" +#include "duckdb/common/operator/multiply.hpp" +#include "duckdb/common/operator/subtract.hpp" #include "duckdb/function/compression/compression.hpp" #include "duckdb/function/compression_function.hpp" #include "duckdb/main/config.hpp" #include "duckdb/storage/buffer_manager.hpp" - +#include "duckdb/storage/compression/bitpacking.hpp" #include "duckdb/storage/table/column_data_checkpointer.hpp" #include "duckdb/storage/table/column_segment.hpp" -#include "duckdb/common/operator/subtract.hpp" -#include "duckdb/common/operator/multiply.hpp" -#include "duckdb/common/operator/add.hpp" -#include "duckdb/storage/compression/bitpacking.hpp" #include "duckdb/storage/table/scan_state.hpp" -#include "duckdb/common/numeric_utils.hpp" #include @@ -63,7 +62,7 @@ typedef struct { typedef uint32_t bitpacking_metadata_encoded_t; static bitpacking_metadata_encoded_t EncodeMeta(bitpacking_metadata_t metadata) { - D_ASSERT(metadata.offset <= 16777215); // max uint24_t + D_ASSERT(metadata.offset <= 0x00FFFFFF); // max uint24_t bitpacking_metadata_encoded_t encoded_value = metadata.offset; encoded_value |= (uint8_t)metadata.mode << 24; return encoded_value; @@ -218,8 +217,11 @@ struct BitpackingState { template void SubtractFrameOfReference(T_INNER *buffer, T_INNER frame_of_reference) { static_assert(IsIntegral::value, "Integral type required."); + + using T_U = typename MakeUnsigned::type; + for (idx_t i = 0; i < compression_buffer_idx; i++) { - buffer[i] -= static_cast::type>(frame_of_reference); + reinterpret_cast(buffer)[i] -= static_cast(frame_of_reference); } } @@ -250,10 +252,8 @@ struct BitpackingState { } // Check if delta has benefit - // bitwidth is calculated differently between signed and unsigned values, but considering we do not have - // an unsigned version of hugeint, we need to explicitly specify (through boolean) that we wish to calculate - // the unsigned minimum bit-width instead of relying on MakeUnsigned and IsSigned - auto delta_required_bitwidth = BitpackingPrimitives::MinimumBitWidth(min_max_delta_diff); + auto delta_required_bitwidth = + BitpackingPrimitives::MinimumBitWidth(static_cast(min_max_delta_diff)); auto regular_required_bitwidth = BitpackingPrimitives::MinimumBitWidth(min_max_diff); if (delta_required_bitwidth < regular_required_bitwidth && mode != BitpackingMode::FOR) { @@ -331,7 +331,7 @@ unique_ptr BitpackingInitAnalyze(ColumnData &col_data, PhysicalTyp template bool BitpackingAnalyze(AnalyzeState &state, Vector &input, idx_t count) { - auto &analyze_state = static_cast &>(state); + auto &analyze_state = state.Cast>(); UnifiedVectorFormat vdata; input.ToUnifiedFormat(count, vdata); @@ -347,7 +347,7 @@ bool BitpackingAnalyze(AnalyzeState &state, Vector &input, idx_t count) { template idx_t BitpackingFinalAnalyze(AnalyzeState &state) { - auto &bitpacking_state = static_cast &>(state); + auto &bitpacking_state = state.Cast>(); auto flush_result = bitpacking_state.state.template Flush(); if (!flush_result) { return DConstants::INVALID_INDEX; @@ -547,7 +547,7 @@ unique_ptr BitpackingInitCompression(ColumnDataCheckpointer &c template void BitpackingCompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { - auto &state = static_cast &>(state_p); + auto &state = state_p.Cast>(); UnifiedVectorFormat vdata; scan_vector.ToUnifiedFormat(count, vdata); state.Append(vdata, count); @@ -555,7 +555,7 @@ void BitpackingCompress(CompressionState &state_p, Vector &scan_vector, idx_t co template void BitpackingFinalizeCompress(CompressionState &state_p) { - auto &state = static_cast &>(state_p); + auto &state = state_p.Cast>(); state.Finalize(); } @@ -690,55 +690,65 @@ struct BitpackingScanState : public SegmentScanState { bool skip_sign_extend = true; idx_t skipped = 0; - while (skipped < skip_count) { - // Exhausted this metadata group, move pointers to next group and load metadata for next group. - if (current_group_offset >= BITPACKING_METADATA_GROUP_SIZE) { - LoadNextGroup(); - } - - idx_t offset_in_compression_group = - current_group_offset % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE; - - if (current_group.mode == BitpackingMode::CONSTANT) { - idx_t remaining = skip_count - skipped; - idx_t to_skip = MinValue(remaining, BITPACKING_METADATA_GROUP_SIZE - current_group_offset); - skipped += to_skip; - current_group_offset += to_skip; - continue; - } - if (current_group.mode == BitpackingMode::CONSTANT_DELTA) { - idx_t remaining = skip_count - skipped; - idx_t to_skip = MinValue(remaining, BITPACKING_METADATA_GROUP_SIZE - current_group_offset); - skipped += to_skip; - current_group_offset += to_skip; - continue; - } - D_ASSERT(current_group.mode == BitpackingMode::FOR || current_group.mode == BitpackingMode::DELTA_FOR); - - idx_t to_skip = - MinValue(skip_count - skipped, - BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE - offset_in_compression_group); - // Calculate start of compression algorithm group - if (current_group.mode == BitpackingMode::DELTA_FOR) { + idx_t initial_group_offset = current_group_offset; + + // This skips straight to the correct metadata group + idx_t meta_groups_to_skip = (skip_count + current_group_offset) / BITPACKING_METADATA_GROUP_SIZE; + if (meta_groups_to_skip) { + + // bitpacking_metadata_ptr points to the next metadata: this means we need to advance the pointer by n-1 + bitpacking_metadata_ptr -= (meta_groups_to_skip - 1) * sizeof(bitpacking_metadata_encoded_t); + LoadNextGroup(); + // The first (partial) group we skipped + skipped += BITPACKING_METADATA_GROUP_SIZE - initial_group_offset; + // The remaining groups that were skipped + skipped += (meta_groups_to_skip - 1) * BITPACKING_METADATA_GROUP_SIZE; + } + + // Assert we can are in the correct metadata group + idx_t remaining_to_skip = skip_count - skipped; + D_ASSERT(current_group_offset + remaining_to_skip < BITPACKING_METADATA_GROUP_SIZE); + + if (current_group.mode == BitpackingMode::CONSTANT || current_group.mode == BitpackingMode::CONSTANT_DELTA || + current_group.mode == BitpackingMode::FOR) { + // Skipping within a constant or constant delta is done by increasing the current_group_offset + skipped += remaining_to_skip; + current_group_offset += remaining_to_skip; + } else { + // For DELTA we actually need to decompress from the current_group_offset up until the row we want to skip + // to this is because we need that delta to be able to continue scanning from here + D_ASSERT(current_group.mode == BitpackingMode::DELTA_FOR); + + while (skipped < skip_count) { + // Calculate compression group offset and pointer + idx_t offset_in_compression_group = + current_group_offset % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE; data_ptr_t current_position_ptr = current_group_ptr + current_group_offset * current_width / 8; data_ptr_t decompression_group_start_pointer = current_position_ptr - offset_in_compression_group * current_width / 8; + idx_t skipping_this_algorithm_group = + MinValue(remaining_to_skip, + BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE - offset_in_compression_group); + BitpackingPrimitives::UnPackBlock(data_ptr_cast(decompression_buffer), decompression_group_start_pointer, current_width, skip_sign_extend); T *decompression_ptr = decompression_buffer + offset_in_compression_group; ApplyFrameOfReference(reinterpret_cast(decompression_ptr), - static_cast(current_frame_of_reference), to_skip); + static_cast(current_frame_of_reference), skipping_this_algorithm_group); DeltaDecode(reinterpret_cast(decompression_ptr), static_cast(current_delta_offset), - to_skip); - current_delta_offset = decompression_ptr[to_skip - 1]; - } + skipping_this_algorithm_group); + current_delta_offset = decompression_ptr[skipping_this_algorithm_group - 1]; - skipped += to_skip; - current_group_offset += to_skip; + skipped += skipping_this_algorithm_group; + current_group_offset += skipping_this_algorithm_group; + remaining_to_skip -= skipping_this_algorithm_group; + } } + + D_ASSERT(skipped == skip_count); } data_ptr_t GetPtr(bitpacking_metadata_t group) { @@ -758,7 +768,7 @@ unique_ptr BitpackingInitScan(ColumnSegment &segment) { template ::type> void BitpackingScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t scan_count, Vector &result, idx_t result_offset) { - auto &scan_state = static_cast &>(*state.scan_state); + auto &scan_state = state.scan_state->Cast>(); T *result_data = FlatVector::GetData(result); result.SetVectorType(VectorType::FLAT_VECTOR); @@ -768,8 +778,10 @@ void BitpackingScanPartial(ColumnSegment &segment, ColumnScanState &state, idx_t idx_t scanned = 0; while (scanned < scan_count) { + D_ASSERT(scan_state.current_group_offset <= BITPACKING_METADATA_GROUP_SIZE); + // Exhausted this metadata group, move pointers to next group and load metadata for next group. - if (scan_state.current_group_offset >= BITPACKING_METADATA_GROUP_SIZE) { + if (scan_state.current_group_offset == BITPACKING_METADATA_GROUP_SIZE) { scan_state.LoadNextGroup(); } @@ -855,6 +867,10 @@ void BitpackingFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t r idx_t result_idx) { BitpackingScanState scan_state(segment); scan_state.Skip(segment, row_id); + + D_ASSERT(scan_state.current_group_offset < BITPACKING_METADATA_GROUP_SIZE); + + D_ASSERT(result.GetVectorType() == VectorType::FLAT_VECTOR); T *result_data = FlatVector::GetData(result); T *current_result_ptr = result_data + result_idx; @@ -900,6 +916,7 @@ void BitpackingFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t r *current_result_ptr += scan_state.current_delta_offset; } } + template void BitpackingSkip(ColumnSegment &segment, ColumnScanState &state, idx_t skip_count) { auto &scan_state = static_cast &>(*state.scan_state); @@ -939,6 +956,8 @@ CompressionFunction BitpackingFun::GetFunction(PhysicalType type) { return GetBitpackingFunction(type); case PhysicalType::INT128: return GetBitpackingFunction(type); + case PhysicalType::UINT128: + return GetBitpackingFunction(type); case PhysicalType::LIST: return GetBitpackingFunction(type); default: @@ -947,6 +966,15 @@ CompressionFunction BitpackingFun::GetFunction(PhysicalType type) { } bool BitpackingFun::TypeIsSupported(PhysicalType type) { + + // we calculate on BITPACKING_METADATA_GROUP_SIZE tuples, but they can exceed the block size, + // in which case we have to disable bitpacking for that data type + // we are conservative here by multiplying by 2 + auto type_size = GetTypeIdSize(type); + if (type_size * BITPACKING_METADATA_GROUP_SIZE * 2 > Storage::BLOCK_SIZE) { + return false; + } + switch (type) { case PhysicalType::BOOL: case PhysicalType::INT8: @@ -959,6 +987,7 @@ bool BitpackingFun::TypeIsSupported(PhysicalType type) { case PhysicalType::UINT64: case PhysicalType::LIST: case PhysicalType::INT128: + case PhysicalType::UINT128: return true; default: return false; diff --git a/src/duckdb/src/storage/compression/bitpacking_hugeint.cpp b/src/duckdb/src/storage/compression/bitpacking_hugeint.cpp index 6743ecd80..3c84c6ec6 100644 --- a/src/duckdb/src/storage/compression/bitpacking_hugeint.cpp +++ b/src/duckdb/src/storage/compression/bitpacking_hugeint.cpp @@ -6,110 +6,110 @@ namespace duckdb { // Unpacking //===--------------------------------------------------------------------===// -static void UnpackSingle(const uint32_t *__restrict &in, hugeint_t *__restrict out, uint16_t delta, uint16_t shr) { +static void UnpackSingle(const uint32_t *__restrict &in, uhugeint_t *__restrict out, uint16_t delta, uint16_t shr) { if (delta + shr < 32) { - *out = ((static_cast(in[0])) >> shr) % (hugeint_t(1) << delta); + *out = ((static_cast(in[0])) >> shr) % (uhugeint_t(1) << delta); } else if (delta + shr >= 32 && delta + shr < 64) { - *out = static_cast(in[0]) >> shr; + *out = static_cast(in[0]) >> shr; ++in; if (delta + shr > 32) { const uint16_t NEXT_SHR = shr + delta - 32; - *out |= static_cast((*in) % (1U << NEXT_SHR)) << (32 - shr); + *out |= static_cast((*in) % (1U << NEXT_SHR)) << (32 - shr); } } else if (delta + shr >= 64 && delta + shr < 96) { - *out = static_cast(in[0]) >> shr; - *out |= static_cast(in[1]) << (32 - shr); + *out = static_cast(in[0]) >> shr; + *out |= static_cast(in[1]) << (32 - shr); in += 2; if (delta + shr > 64) { const uint16_t NEXT_SHR = delta + shr - 64; - *out |= static_cast((*in) % (1U << NEXT_SHR)) << (64 - shr); + *out |= static_cast((*in) % (1U << NEXT_SHR)) << (64 - shr); } } else if (delta + shr >= 96 && delta + shr < 128) { - *out = static_cast(in[0]) >> shr; - *out |= static_cast(in[1]) << (32 - shr); - *out |= static_cast(in[2]) << (64 - shr); + *out = static_cast(in[0]) >> shr; + *out |= static_cast(in[1]) << (32 - shr); + *out |= static_cast(in[2]) << (64 - shr); in += 3; if (delta + shr > 96) { const uint16_t NEXT_SHR = delta + shr - 96; - *out |= static_cast((*in) % (1U << NEXT_SHR)) << (96 - shr); + *out |= static_cast((*in) % (1U << NEXT_SHR)) << (96 - shr); } } else if (delta + shr >= 128) { - *out = static_cast(in[0]) >> shr; - *out |= static_cast(in[1]) << (32 - shr); - *out |= static_cast(in[2]) << (64 - shr); - *out |= static_cast(in[3]) << (96 - shr); + *out = static_cast(in[0]) >> shr; + *out |= static_cast(in[1]) << (32 - shr); + *out |= static_cast(in[2]) << (64 - shr); + *out |= static_cast(in[3]) << (96 - shr); in += 4; if (delta + shr > 128) { const uint16_t NEXT_SHR = delta + shr - 128; - *out |= static_cast((*in) % (1U << NEXT_SHR)) << (128 - shr); + *out |= static_cast((*in) % (1U << NEXT_SHR)) << (128 - shr); } } } -static void UnpackLast(const uint32_t *__restrict &in, hugeint_t *__restrict out, uint16_t delta) { +static void UnpackLast(const uint32_t *__restrict &in, uhugeint_t *__restrict out, uint16_t delta) { const uint8_t LAST_IDX = 31; const uint16_t SHIFT = (delta * 31) % 32; out[LAST_IDX] = in[0] >> SHIFT; if (delta > 32) { - out[LAST_IDX] |= static_cast(in[1]) << (32 - SHIFT); + out[LAST_IDX] |= static_cast(in[1]) << (32 - SHIFT); } if (delta > 64) { - out[LAST_IDX] |= static_cast(in[2]) << (64 - SHIFT); + out[LAST_IDX] |= static_cast(in[2]) << (64 - SHIFT); } if (delta > 96) { - out[LAST_IDX] |= static_cast(in[3]) << (96 - SHIFT); + out[LAST_IDX] |= static_cast(in[3]) << (96 - SHIFT); } } // Unpacks for specific deltas -static void UnpackDelta0(const uint32_t *__restrict in, hugeint_t *__restrict out) { +static void UnpackDelta0(const uint32_t *__restrict in, uhugeint_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { out[i] = 0; } } -static void UnpackDelta32(const uint32_t *__restrict in, hugeint_t *__restrict out) { +static void UnpackDelta32(const uint32_t *__restrict in, uhugeint_t *__restrict out) { for (uint8_t k = 0; k < 32; ++k) { - out[k] = static_cast(in[k]); + out[k] = static_cast(in[k]); } } -static void UnpackDelta64(const uint32_t *__restrict in, hugeint_t *__restrict out) { +static void UnpackDelta64(const uint32_t *__restrict in, uhugeint_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = i * 2; out[i] = in[OFFSET]; - out[i] |= static_cast(in[OFFSET + 1]) << 32; + out[i] |= static_cast(in[OFFSET + 1]) << 32; } } -static void UnpackDelta96(const uint32_t *__restrict in, hugeint_t *__restrict out) { +static void UnpackDelta96(const uint32_t *__restrict in, uhugeint_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = i * 3; out[i] = in[OFFSET]; - out[i] |= static_cast(in[OFFSET + 1]) << 32; - out[i] |= static_cast(in[OFFSET + 2]) << 64; + out[i] |= static_cast(in[OFFSET + 1]) << 32; + out[i] |= static_cast(in[OFFSET + 2]) << 64; } } -static void UnpackDelta128(const uint32_t *__restrict in, hugeint_t *__restrict out) { +static void UnpackDelta128(const uint32_t *__restrict in, uhugeint_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = i * 4; out[i] = in[OFFSET]; - out[i] |= static_cast(in[OFFSET + 1]) << 32; - out[i] |= static_cast(in[OFFSET + 2]) << 64; - out[i] |= static_cast(in[OFFSET + 3]) << 96; + out[i] |= static_cast(in[OFFSET + 1]) << 32; + out[i] |= static_cast(in[OFFSET + 2]) << 64; + out[i] |= static_cast(in[OFFSET + 3]) << 96; } } @@ -117,7 +117,7 @@ static void UnpackDelta128(const uint32_t *__restrict in, hugeint_t *__restrict // Packing //===--------------------------------------------------------------------===// -static void PackSingle(const hugeint_t in, uint32_t *__restrict &out, uint16_t delta, uint16_t shl, hugeint_t mask) { +static void PackSingle(const uhugeint_t in, uint32_t *__restrict &out, uint16_t delta, uint16_t shl, uhugeint_t mask) { if (delta + shl < 32) { if (shl == 0) { @@ -186,7 +186,7 @@ static void PackSingle(const hugeint_t in, uint32_t *__restrict &out, uint16_t d } } -static void PackLast(const hugeint_t *__restrict in, uint32_t *__restrict out, uint16_t delta) { +static void PackLast(const uhugeint_t *__restrict in, uint32_t *__restrict out, uint16_t delta) { const uint8_t LAST_IDX = 31; const uint16_t SHIFT = (delta * 31) % 32; out[0] |= static_cast(in[LAST_IDX] << SHIFT); @@ -202,13 +202,13 @@ static void PackLast(const hugeint_t *__restrict in, uint32_t *__restrict out, u } // Packs for specific deltas -static void PackDelta32(const hugeint_t *__restrict in, uint32_t *__restrict out) { +static void PackDelta32(const uhugeint_t *__restrict in, uint32_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { out[i] = static_cast(in[i]); } } -static void PackDelta64(const hugeint_t *__restrict in, uint32_t *__restrict out) { +static void PackDelta64(const uhugeint_t *__restrict in, uint32_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = 2 * i; out[OFFSET] = static_cast(in[i]); @@ -216,7 +216,7 @@ static void PackDelta64(const hugeint_t *__restrict in, uint32_t *__restrict out } } -static void PackDelta96(const hugeint_t *__restrict in, uint32_t *__restrict out) { +static void PackDelta96(const uhugeint_t *__restrict in, uint32_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = 3 * i; out[OFFSET] = static_cast(in[i]); @@ -225,7 +225,7 @@ static void PackDelta96(const hugeint_t *__restrict in, uint32_t *__restrict out } } -static void PackDelta128(const hugeint_t *__restrict in, uint32_t *__restrict out) { +static void PackDelta128(const uhugeint_t *__restrict in, uint32_t *__restrict out) { for (uint8_t i = 0; i < 32; ++i) { const uint8_t OFFSET = 4 * i; out[OFFSET] = static_cast(in[i]); @@ -239,7 +239,7 @@ static void PackDelta128(const hugeint_t *__restrict in, uint32_t *__restrict ou // HugeIntPacker //===--------------------------------------------------------------------===// -void HugeIntPacker::Pack(const hugeint_t *__restrict in, uint32_t *__restrict out, bitpacking_width_t width) { +void HugeIntPacker::Pack(const uhugeint_t *__restrict in, uint32_t *__restrict out, bitpacking_width_t width) { D_ASSERT(width <= 128); switch (width) { case 0: @@ -259,13 +259,13 @@ void HugeIntPacker::Pack(const hugeint_t *__restrict in, uint32_t *__restrict ou default: for (idx_t oindex = 0; oindex < BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE - 1; ++oindex) { PackSingle(in[oindex], out, width, (width * oindex) % BitpackingPrimitives::BITPACKING_ALGORITHM_GROUP_SIZE, - (hugeint_t(1) << width) - 1); + (uhugeint_t(1) << width) - 1); } PackLast(in, out, width); } } -void HugeIntPacker::Unpack(const uint32_t *__restrict in, hugeint_t *__restrict out, bitpacking_width_t width) { +void HugeIntPacker::Unpack(const uint32_t *__restrict in, uhugeint_t *__restrict out, bitpacking_width_t width) { D_ASSERT(width <= 128); switch (width) { case 0: diff --git a/src/duckdb/src/storage/compression/dictionary_compression.cpp b/src/duckdb/src/storage/compression/dictionary_compression.cpp index 1c74c102e..b5d071843 100644 --- a/src/duckdb/src/storage/compression/dictionary_compression.cpp +++ b/src/duckdb/src/storage/compression/dictionary_compression.cpp @@ -4,10 +4,8 @@ #include "duckdb/common/types/vector_buffer.hpp" #include "duckdb/function/compression/compression.hpp" #include "duckdb/function/compression_function.hpp" -#include "duckdb/main/config.hpp" #include "duckdb/storage/segment/uncompressed.hpp" #include "duckdb/storage/string_uncompressed.hpp" -#include "duckdb/storage/table/append_state.hpp" #include "duckdb/storage/table/column_data_checkpointer.hpp" namespace duckdb { @@ -612,7 +610,7 @@ void DictionaryCompressionStorage::SetDictionary(ColumnSegment &segment, BufferH string_t DictionaryCompressionStorage::FetchStringFromDict(ColumnSegment &segment, StringDictionaryContainer dict, data_ptr_t baseptr, int32_t dict_offset, uint16_t string_len) { - D_ASSERT(dict_offset >= 0 && dict_offset <= Storage::BLOCK_SIZE); + D_ASSERT(dict_offset >= 0 && dict_offset <= int32_t(Storage::BLOCK_SIZE)); if (dict_offset == 0) { return string_t(nullptr, 0); diff --git a/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp b/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp index ea77f9269..c1b210d78 100644 --- a/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp +++ b/src/duckdb/src/storage/compression/fixed_size_uncompressed.cpp @@ -284,6 +284,8 @@ CompressionFunction FixedSizeUncompressed::GetFunction(PhysicalType data_type) { return FixedSizeGetFunction(data_type); case PhysicalType::INT128: return FixedSizeGetFunction(data_type); + case PhysicalType::UINT128: + return FixedSizeGetFunction(data_type); case PhysicalType::FLOAT: return FixedSizeGetFunction(data_type); case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/storage/compression/fsst.cpp b/src/duckdb/src/storage/compression/fsst.cpp index 9729af274..f5809653c 100644 --- a/src/duckdb/src/storage/compression/fsst.cpp +++ b/src/duckdb/src/storage/compression/fsst.cpp @@ -392,17 +392,17 @@ class FSSTCompressionState : public CompressionState { unique_ptr FSSTStorage::InitCompression(ColumnDataCheckpointer &checkpointer, unique_ptr analyze_state_p) { - auto analyze_state = static_cast(analyze_state_p.get()); + auto &analyze_state = analyze_state_p->Cast(); auto compression_state = make_uniq(checkpointer); - if (analyze_state->fsst_encoder == nullptr) { + if (analyze_state.fsst_encoder == nullptr) { throw InternalException("No encoder found during FSST compression"); } - compression_state->fsst_encoder = analyze_state->fsst_encoder; + compression_state->fsst_encoder = analyze_state.fsst_encoder; compression_state->fsst_serialized_symbol_table_size = duckdb_fsst_export(compression_state->fsst_encoder, &compression_state->fsst_serialized_symbol_table[0]); - analyze_state->fsst_encoder = nullptr; + analyze_state.fsst_encoder = nullptr; return std::move(compression_state); } diff --git a/src/duckdb/src/storage/compression/numeric_constant.cpp b/src/duckdb/src/storage/compression/numeric_constant.cpp index bb18e1b85..3741f71a3 100644 --- a/src/duckdb/src/storage/compression/numeric_constant.cpp +++ b/src/duckdb/src/storage/compression/numeric_constant.cpp @@ -128,6 +128,8 @@ CompressionFunction ConstantFun::GetFunction(PhysicalType data_type) { return ConstantGetFunction(data_type); case PhysicalType::INT128: return ConstantGetFunction(data_type); + case PhysicalType::UINT128: + return ConstantGetFunction(data_type); case PhysicalType::FLOAT: return ConstantGetFunction(data_type); case PhysicalType::DOUBLE: @@ -150,6 +152,7 @@ bool ConstantFun::TypeIsSupported(PhysicalType type) { case PhysicalType::UINT32: case PhysicalType::UINT64: case PhysicalType::INT128: + case PhysicalType::UINT128: case PhysicalType::FLOAT: case PhysicalType::DOUBLE: return true; diff --git a/src/duckdb/src/storage/compression/rle.cpp b/src/duckdb/src/storage/compression/rle.cpp index a79440919..89a800b80 100644 --- a/src/duckdb/src/storage/compression/rle.cpp +++ b/src/duckdb/src/storage/compression/rle.cpp @@ -132,8 +132,7 @@ struct RLECompressState : public CompressionState { static idx_t MaxRLECount() { auto entry_size = sizeof(T) + sizeof(rle_count_t); auto entry_count = (Storage::BLOCK_SIZE - RLEConstants::RLE_HEADER_SIZE) / entry_size; - auto max_vector_count = entry_count / STANDARD_VECTOR_SIZE; - return max_vector_count * STANDARD_VECTOR_SIZE; + return entry_count; } explicit RLECompressState(ColumnDataCheckpointer &checkpointer_p) @@ -228,7 +227,7 @@ unique_ptr RLEInitCompression(ColumnDataCheckpointer &checkpoi template void RLECompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { - auto &state = (RLECompressState &)state_p; + auto &state = state_p.Cast>(); UnifiedVectorFormat vdata; scan_vector.ToUnifiedFormat(count, vdata); @@ -237,7 +236,7 @@ void RLECompress(CompressionState &state_p, Vector &scan_vector, idx_t count) { template void RLEFinalizeCompress(CompressionState &state_p) { - auto &state = (RLECompressState &)state_p; + auto &state = state_p.Cast>(); state.Finalize(); } @@ -411,6 +410,8 @@ CompressionFunction RLEFun::GetFunction(PhysicalType type) { return GetRLEFunction(type); case PhysicalType::INT128: return GetRLEFunction(type); + case PhysicalType::UINT128: + return GetRLEFunction(type); case PhysicalType::UINT8: return GetRLEFunction(type); case PhysicalType::UINT16: @@ -442,6 +443,7 @@ bool RLEFun::TypeIsSupported(PhysicalType type) { case PhysicalType::UINT16: case PhysicalType::UINT32: case PhysicalType::UINT64: + case PhysicalType::UINT128: case PhysicalType::FLOAT: case PhysicalType::DOUBLE: case PhysicalType::LIST: diff --git a/src/duckdb/src/storage/compression/string_uncompressed.cpp b/src/duckdb/src/storage/compression/string_uncompressed.cpp index 4f7c6da24..fd8dfce01 100644 --- a/src/duckdb/src/storage/compression/string_uncompressed.cpp +++ b/src/duckdb/src/storage/compression/string_uncompressed.cpp @@ -110,8 +110,8 @@ BufferHandle &ColumnFetchState::GetOrInsertHandle(ColumnSegment &segment) { // not pinned yet: pin it auto &buffer_manager = BufferManager::GetBufferManager(segment.db); auto handle = buffer_manager.Pin(segment.block); - auto entry = handles.insert(make_pair(primary_id, std::move(handle))); - return entry.first->second; + auto pinned_entry = handles.insert(make_pair(primary_id, std::move(handle))); + return pinned_entry.first->second; } else { // already pinned: use the pinned handle return entry->second; @@ -322,7 +322,7 @@ void UncompressedStringStorage::WriteStringMemory(ColumnSegment &segment, string string_t UncompressedStringStorage::ReadOverflowString(ColumnSegment &segment, Vector &result, block_id_t block, int32_t offset) { D_ASSERT(block != INVALID_BLOCK); - D_ASSERT(offset < Storage::BLOCK_SIZE); + D_ASSERT(offset < int32_t(Storage::BLOCK_SIZE)); auto &block_manager = segment.GetBlockManager(); auto &buffer_manager = block_manager.buffer_manager; @@ -403,7 +403,7 @@ void UncompressedStringStorage::ReadStringMarker(data_ptr_t target, block_id_t & string_location_t UncompressedStringStorage::FetchStringLocation(StringDictionaryContainer dict, data_ptr_t baseptr, int32_t dict_offset) { - D_ASSERT(dict_offset >= -1 * Storage::BLOCK_SIZE && dict_offset <= Storage::BLOCK_SIZE); + D_ASSERT(dict_offset >= -1 * int32_t(Storage::BLOCK_SIZE) && dict_offset <= int32_t(Storage::BLOCK_SIZE)); if (dict_offset < 0) { string_location_t result; ReadStringMarker(baseptr + dict.end - (-1 * dict_offset), result.block_id, result.offset); @@ -417,7 +417,7 @@ string_t UncompressedStringStorage::FetchStringFromDict(ColumnSegment &segment, Vector &result, data_ptr_t baseptr, int32_t dict_offset, uint32_t string_length) { // fetch base data - D_ASSERT(dict_offset <= Storage::BLOCK_SIZE); + D_ASSERT(dict_offset <= int32_t(Storage::BLOCK_SIZE)); string_location_t location = FetchStringLocation(dict, baseptr, dict_offset); return FetchString(segment, dict, result, baseptr, location, string_length); } diff --git a/src/duckdb/src/storage/compression/uncompressed.cpp b/src/duckdb/src/storage/compression/uncompressed.cpp index 9955bf8a6..b68c4fce0 100644 --- a/src/duckdb/src/storage/compression/uncompressed.cpp +++ b/src/duckdb/src/storage/compression/uncompressed.cpp @@ -15,6 +15,7 @@ CompressionFunction UncompressedFun::GetFunction(PhysicalType type) { case PhysicalType::UINT16: case PhysicalType::UINT32: case PhysicalType::UINT64: + case PhysicalType::UINT128: case PhysicalType::FLOAT: case PhysicalType::DOUBLE: case PhysicalType::LIST: diff --git a/src/duckdb/src/storage/data_table.cpp b/src/duckdb/src/storage/data_table.cpp index a5e619048..0e64ab831 100644 --- a/src/duckdb/src/storage/data_table.cpp +++ b/src/duckdb/src/storage/data_table.cpp @@ -19,12 +19,12 @@ #include "duckdb/storage/table/standard_column_data.hpp" #include "duckdb/transaction/duck_transaction.hpp" #include "duckdb/transaction/transaction_manager.hpp" -#include "duckdb/execution/index/art/art.hpp" #include "duckdb/main/attached_database.hpp" #include "duckdb/common/types/conflict_manager.hpp" #include "duckdb/common/types/constraint_conflict_info.hpp" #include "duckdb/storage/table/append_state.hpp" #include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -34,6 +34,10 @@ DataTableInfo::DataTableInfo(AttachedDatabase &db, shared_ptr ta table(std::move(table)) { } +void DataTableInfo::InitializeIndexes(ClientContext &context) { + indexes.InitializeIndexes(context, *this); +} + bool DataTableInfo::IsTemporary() const { return db.IsTemporary(); } @@ -84,6 +88,9 @@ DataTable::DataTable(ClientContext &context, DataTable &parent, idx_t removed_co for (auto &column_def : parent.column_definitions) { column_definitions.emplace_back(column_def.Copy()); } + + // try to initialize unknown indexes + info->InitializeIndexes(context); // first check if there are any indexes that exist that point to the removed column info->indexes.Scan([&](Index &index) { for (auto &column_id : index.column_ids) { @@ -148,6 +155,9 @@ DataTable::DataTable(ClientContext &context, DataTable &parent, idx_t changed_id for (auto &column_def : parent.column_definitions) { column_definitions.emplace_back(column_def.Copy()); } + // try to initialize unknown indexes + info->InitializeIndexes(context); + // first check if there are any indexes that exist that point to the changed column info->indexes.Scan([&](Index &index) { for (auto &column_id : index.column_ids) { @@ -288,8 +298,9 @@ static void VerifyGeneratedExpressionSuccess(ClientContext &context, TableCatalo } catch (InternalException &ex) { throw; } catch (std::exception &ex) { + ErrorData error(ex); throw ConstraintException("Incorrect value for generated column \"%s %s AS (%s)\" : %s", col.Name(), - col.Type().ToString(), col.GeneratedExpression().ToString(), ex.what()); + col.Type().ToString(), col.GeneratedExpression().ToString(), error.RawMessage()); } } @@ -300,7 +311,8 @@ static void VerifyCheckConstraint(ClientContext &context, TableCatalogEntry &tab try { executor.ExecuteExpression(chunk, result); } catch (std::exception &ex) { - throw ConstraintException("CHECK constraint failed: %s (Error: %s)", table.name, ex.what()); + ErrorData error(ex); + throw ConstraintException("CHECK constraint failed: %s (Error: %s)", table.name, error.RawMessage()); } catch (...) { // LCOV_EXCL_START throw ConstraintException("CHECK constraint failed: %s (Unknown Error)", table.name); } // LCOV_EXCL_STOP @@ -369,13 +381,9 @@ idx_t LocateErrorIndex(bool is_append, const ManagedSelection &matches) { DataChunk &input) { auto verify_type = is_append ? VerifyExistenceType::APPEND_FK : VerifyExistenceType::DELETE_FK; - D_ASSERT(failed_index != DConstants::INVALID_INDEX); - D_ASSERT(index.index_type == "ART"); - auto &art_index = index.Cast(); - auto key_name = art_index.GenerateErrorKeyName(input, failed_index); - auto exception_msg = art_index.GenerateConstraintErrorMessage(verify_type, key_name); - throw ConstraintException(exception_msg); + auto message = index.GetConstraintViolationMessage(verify_type, failed_index, input); + throw ConstraintException(message); } bool IsForeignKeyConstraintError(bool is_append, idx_t input_count, const ManagedSelection &matches) { @@ -868,8 +876,8 @@ void DataTable::RevertAppend(idx_t start_row, idx_t count) { //===--------------------------------------------------------------------===// // Indexes //===--------------------------------------------------------------------===// -PreservedError DataTable::AppendToIndexes(TableIndexList &indexes, DataChunk &chunk, row_t row_start) { - PreservedError error; +ErrorData DataTable::AppendToIndexes(TableIndexList &indexes, DataChunk &chunk, row_t row_start) { + ErrorData error; if (indexes.Empty()) { return error; } @@ -883,12 +891,10 @@ PreservedError DataTable::AppendToIndexes(TableIndexList &indexes, DataChunk &ch indexes.Scan([&](Index &index) { try { error = index.Append(chunk, row_identifiers); - } catch (Exception &ex) { - error = PreservedError(ex); } catch (std::exception &ex) { - error = PreservedError(ex); + error = ErrorData(ex); } - if (error) { + if (error.HasError()) { append_failed = true; return true; } @@ -906,7 +912,7 @@ PreservedError DataTable::AppendToIndexes(TableIndexList &indexes, DataChunk &ch return error; } -PreservedError DataTable::AppendToIndexes(DataChunk &chunk, row_t row_start) { +ErrorData DataTable::AppendToIndexes(DataChunk &chunk, row_t row_start) { D_ASSERT(is_root); return AppendToIndexes(info->indexes, chunk, row_start); } diff --git a/src/duckdb/src/storage/index.cpp b/src/duckdb/src/storage/index.cpp index 6b554d1d8..01b2b6c5f 100644 --- a/src/duckdb/src/storage/index.cpp +++ b/src/duckdb/src/storage/index.cpp @@ -39,7 +39,7 @@ void Index::InitializeLock(IndexLock &state) { state.index_lock = unique_lock(lock); } -PreservedError Index::Append(DataChunk &entries, Vector &row_identifiers) { +ErrorData Index::Append(DataChunk &entries, Vector &row_identifiers) { IndexLock state; InitializeLock(state); return Append(state, entries, row_identifiers); diff --git a/src/duckdb/src/storage/local_storage.cpp b/src/duckdb/src/storage/local_storage.cpp index b1e856364..2aa7ab736 100644 --- a/src/duckdb/src/storage/local_storage.cpp +++ b/src/duckdb/src/storage/local_storage.cpp @@ -23,7 +23,11 @@ LocalTableStorage::LocalTableStorage(DataTable &table) row_groups->InitializeEmpty(); table.info->indexes.Scan([&](Index &index) { - D_ASSERT(index.index_type == "ART"); + if (index.index_type != ART::TYPE_NAME) { + return false; + } + D_ASSERT(index.index_type == ART::TYPE_NAME); + auto &art = index.Cast(); if (art.index_constraint_type != IndexConstraintType::NONE) { // unique index: create a local ART index that maintains the same unique constraint @@ -116,16 +120,16 @@ void LocalTableStorage::FlushBlocks() { optimistic_writer.FinalFlush(); } -PreservedError LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, RowGroupCollection &source, - TableIndexList &index_list, const vector &table_types, - row_t &start_row) { +ErrorData LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, RowGroupCollection &source, + TableIndexList &index_list, const vector &table_types, + row_t &start_row) { // only need to scan for index append // figure out which columns we need to scan for the set of indexes auto columns = index_list.GetRequiredColumns(); // create an empty mock chunk that contains all the correct types for the table DataChunk mock_chunk; mock_chunk.InitializeEmpty(table_types); - PreservedError error; + ErrorData error; source.Scan(transaction, columns, [&](DataChunk &chunk) -> bool { // construct the mock chunk by referencing the required columns for (idx_t i = 0; i < columns.size(); i++) { @@ -134,7 +138,7 @@ PreservedError LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, mock_chunk.SetCardinality(chunk); // append this chunk to the indexes of the table error = DataTable::AppendToIndexes(index_list, mock_chunk, start_row); - if (error) { + if (error.HasError()) { return false; } start_row += chunk.size(); @@ -149,13 +153,13 @@ void LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, TableAppen if (append_to_table) { table.InitializeAppend(transaction, append_state, append_count); } - PreservedError error; + ErrorData error; if (append_to_table) { // appending: need to scan entire row_groups->Scan(transaction, [&](DataChunk &chunk) -> bool { // append this chunk to the indexes of the table error = table.AppendToIndexes(chunk, append_state.current_row); - if (error) { + if (error.HasError()) { return false; } // append to base table @@ -166,7 +170,7 @@ void LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, TableAppen error = AppendToIndexes(transaction, *row_groups, table.info->indexes, table.GetTypes(), append_state.current_row); } - if (error) { + if (error.HasError()) { // need to revert all appended row ids row_t current_row = append_state.row_start; // remove the data from the indexes, if there are any indexes @@ -174,11 +178,8 @@ void LocalTableStorage::AppendToIndexes(DuckTransaction &transaction, TableAppen // append this chunk to the indexes of the table try { table.RemoveFromIndexes(append_state, chunk, current_row); - } catch (Exception &ex) { - error = PreservedError(ex); - return false; } catch (std::exception &ex) { // LCOV_EXCL_START - error = PreservedError(ex); + error = ErrorData(ex); return false; } // LCOV_EXCL_STOP @@ -359,7 +360,7 @@ void LocalStorage::Append(LocalAppendState &state, DataChunk &chunk) { auto storage = state.storage; idx_t base_id = MAX_ROW_ID + storage->row_groups->GetTotalRows() + state.append_state.total_append_count; auto error = DataTable::AppendToIndexes(storage->indexes, chunk, base_id); - if (error) { + if (error.HasError()) { error.Throw(); } @@ -381,7 +382,7 @@ void LocalStorage::LocalMerge(DataTable &table, RowGroupCollection &collection) // append data to indexes if required row_t base_id = MAX_ROW_ID + storage.row_groups->GetTotalRows(); auto error = storage.AppendToIndexes(transaction, collection, storage.indexes, table.GetTypes(), base_id); - if (error) { + if (error.HasError()) { error.Throw(); } } @@ -466,6 +467,9 @@ void LocalStorage::Flush(DataTable &table, LocalTableStorage &storage) { storage.AppendToIndexes(transaction, append_state, append_count, true); } + // try to initialize any unknown indexes + table.info->InitializeIndexes(context); + // possibly vacuum any excess index data table.info->indexes.Scan([&](Index &index) { index.Vacuum(); @@ -568,6 +572,7 @@ TableIndexList &LocalStorage::GetIndexes(DataTable &table) { if (!storage) { throw InternalException("LocalStorage::GetIndexes - local storage not found"); } + table.info->InitializeIndexes(context); return storage->indexes; } diff --git a/src/duckdb/src/storage/partial_block_manager.cpp b/src/duckdb/src/storage/partial_block_manager.cpp index 8128caf01..8a91e8844 100644 --- a/src/duckdb/src/storage/partial_block_manager.cpp +++ b/src/duckdb/src/storage/partial_block_manager.cpp @@ -119,7 +119,7 @@ void PartialBlockManager::RegisterPartialBlock(PartialBlockAllocation &&allocati // Free the page with the least space free. auto itr = partially_filled_blocks.begin(); block_to_free = std::move(itr->second); - free_space = state.block_size - itr->first; + free_space = itr->first; partially_filled_blocks.erase(itr); } // Flush any block that we're not going to reuse. diff --git a/src/duckdb/src/storage/serialization/serialize_create_info.cpp b/src/duckdb/src/storage/serialization/serialize_create_info.cpp index b59cf1458..3631e3197 100644 --- a/src/duckdb/src/storage/serialization/serialize_create_info.cpp +++ b/src/duckdb/src/storage/serialization/serialize_create_info.cpp @@ -24,6 +24,7 @@ void CreateInfo::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(104, "internal", internal); serializer.WriteProperty(105, "on_conflict", on_conflict); serializer.WritePropertyWithDefault(106, "sql", sql); + serializer.WritePropertyWithDefault(107, "comment", comment, Value()); } unique_ptr CreateInfo::Deserialize(Deserializer &deserializer) { @@ -34,6 +35,7 @@ unique_ptr CreateInfo::Deserialize(Deserializer &deserializer) { auto internal = deserializer.ReadPropertyWithDefault(104, "internal"); auto on_conflict = deserializer.ReadProperty(105, "on_conflict"); auto sql = deserializer.ReadPropertyWithDefault(106, "sql"); + auto comment = deserializer.ReadPropertyWithDefault(107, "comment", Value()); deserializer.Set(type); unique_ptr result; switch (type) { @@ -71,6 +73,7 @@ unique_ptr CreateInfo::Deserialize(Deserializer &deserializer) { result->internal = internal; result->on_conflict = on_conflict; result->sql = std::move(sql); + result->comment = comment; return result; } @@ -78,7 +81,7 @@ void CreateIndexInfo::Serialize(Serializer &serializer) const { CreateInfo::Serialize(serializer); serializer.WritePropertyWithDefault(200, "name", index_name); serializer.WritePropertyWithDefault(201, "table", table); - /* [Deleted] (IndexType) "index_type" */ + /* [Deleted] (DeprecatedIndexType) "index_type" */ serializer.WriteProperty(203, "constraint_type", constraint_type); serializer.WritePropertyWithDefault>>(204, "parsed_expressions", parsed_expressions); serializer.WritePropertyWithDefault>(205, "scan_types", scan_types); @@ -92,7 +95,7 @@ unique_ptr CreateIndexInfo::Deserialize(Deserializer &deserializer) auto result = duckdb::unique_ptr(new CreateIndexInfo()); deserializer.ReadPropertyWithDefault(200, "name", result->index_name); deserializer.ReadPropertyWithDefault(201, "table", result->table); - deserializer.ReadDeletedProperty(202, "index_type"); + deserializer.ReadDeletedProperty(202, "index_type"); deserializer.ReadProperty(203, "constraint_type", result->constraint_type); deserializer.ReadPropertyWithDefault>>(204, "parsed_expressions", result->parsed_expressions); deserializer.ReadPropertyWithDefault>(205, "scan_types", result->scan_types); diff --git a/src/duckdb/src/storage/serialization/serialize_extra_drop_info.cpp b/src/duckdb/src/storage/serialization/serialize_extra_drop_info.cpp new file mode 100644 index 000000000..fcdfd4197 --- /dev/null +++ b/src/duckdb/src/storage/serialization/serialize_extra_drop_info.cpp @@ -0,0 +1,42 @@ +//===----------------------------------------------------------------------===// +// This file is automatically generated by scripts/generate_serialization.py +// Do not edit this file manually, your changes will be overwritten +//===----------------------------------------------------------------------===// + +#include "duckdb/common/serializer/serializer.hpp" +#include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/parser/parsed_data/extra_drop_info.hpp" + +namespace duckdb { + +void ExtraDropInfo::Serialize(Serializer &serializer) const { + serializer.WriteProperty(100, "info_type", info_type); +} + +unique_ptr ExtraDropInfo::Deserialize(Deserializer &deserializer) { + auto info_type = deserializer.ReadProperty(100, "info_type"); + unique_ptr result; + switch (info_type) { + case ExtraDropInfoType::SECRET_INFO: + result = ExtraDropSecretInfo::Deserialize(deserializer); + break; + default: + throw SerializationException("Unsupported type for deserialization of ExtraDropInfo!"); + } + return result; +} + +void ExtraDropSecretInfo::Serialize(Serializer &serializer) const { + ExtraDropInfo::Serialize(serializer); + serializer.WriteProperty(200, "persist_mode", persist_mode); + serializer.WritePropertyWithDefault(201, "secret_storage", secret_storage); +} + +unique_ptr ExtraDropSecretInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::unique_ptr(new ExtraDropSecretInfo()); + deserializer.ReadProperty(200, "persist_mode", result->persist_mode); + deserializer.ReadPropertyWithDefault(201, "secret_storage", result->secret_storage); + return std::move(result); +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp index 71c5ac2ef..fd061c1db 100644 --- a/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp +++ b/src/duckdb/src/storage/serialization/serialize_logical_operator.cpp @@ -157,9 +157,6 @@ unique_ptr LogicalOperator::Deserialize(Deserializer &deseriali case LogicalOperatorType::LOGICAL_SET: result = LogicalSet::Deserialize(deserializer); break; - case LogicalOperatorType::LOGICAL_SHOW: - result = LogicalShow::Deserialize(deserializer); - break; case LogicalOperatorType::LOGICAL_TOP_N: result = LogicalTopN::Deserialize(deserializer); break; @@ -288,6 +285,7 @@ void LogicalComparisonJoin::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault>(204, "conditions", conditions); serializer.WritePropertyWithDefault>(205, "mark_types", mark_types); serializer.WritePropertyWithDefault>>(206, "duplicate_eliminated_columns", duplicate_eliminated_columns); + serializer.WritePropertyWithDefault(207, "delim_flipped", delim_flipped, false); } unique_ptr LogicalComparisonJoin::Deserialize(Deserializer &deserializer) { @@ -299,6 +297,7 @@ unique_ptr LogicalComparisonJoin::Deserialize(Deserializer &des deserializer.ReadPropertyWithDefault>(204, "conditions", result->conditions); deserializer.ReadPropertyWithDefault>(205, "mark_types", result->mark_types); deserializer.ReadPropertyWithDefault>>(206, "duplicate_eliminated_columns", result->duplicate_eliminated_columns); + deserializer.ReadPropertyWithDefault(207, "delim_flipped", result->delim_flipped, false); return std::move(result); } @@ -675,19 +674,6 @@ unique_ptr LogicalSetOperation::Deserialize(Deserializer &deser return std::move(result); } -void LogicalShow::Serialize(Serializer &serializer) const { - LogicalOperator::Serialize(serializer); - serializer.WritePropertyWithDefault>(200, "types_select", types_select); - serializer.WritePropertyWithDefault>(201, "aliases", aliases); -} - -unique_ptr LogicalShow::Deserialize(Deserializer &deserializer) { - auto result = duckdb::unique_ptr(new LogicalShow()); - deserializer.ReadPropertyWithDefault>(200, "types_select", result->types_select); - deserializer.ReadPropertyWithDefault>(201, "aliases", result->aliases); - return std::move(result); -} - void LogicalSimple::Serialize(Serializer &serializer) const { LogicalOperator::Serialize(serializer); serializer.WritePropertyWithDefault>(200, "info", info); diff --git a/src/duckdb/src/storage/serialization/serialize_macro_function.cpp b/src/duckdb/src/storage/serialization/serialize_macro_function.cpp index 79484d808..8be5988db 100644 --- a/src/duckdb/src/storage/serialization/serialize_macro_function.cpp +++ b/src/duckdb/src/storage/serialization/serialize_macro_function.cpp @@ -14,13 +14,13 @@ namespace duckdb { void MacroFunction::Serialize(Serializer &serializer) const { serializer.WriteProperty(100, "type", type); serializer.WritePropertyWithDefault>>(101, "parameters", parameters); - serializer.WritePropertyWithDefault>>(102, "default_parameters", default_parameters); + serializer.WritePropertyWithDefault>>(102, "default_parameters", default_parameters); } unique_ptr MacroFunction::Deserialize(Deserializer &deserializer) { auto type = deserializer.ReadProperty(100, "type"); auto parameters = deserializer.ReadPropertyWithDefault>>(101, "parameters"); - auto default_parameters = deserializer.ReadPropertyWithDefault>>(102, "default_parameters"); + auto default_parameters = deserializer.ReadPropertyWithDefault>>(102, "default_parameters"); unique_ptr result; switch (type) { case MacroType::SCALAR_MACRO: diff --git a/src/duckdb/src/storage/serialization/serialize_nodes.cpp b/src/duckdb/src/storage/serialization/serialize_nodes.cpp index 420a26fe6..13df905c9 100644 --- a/src/duckdb/src/storage/serialization/serialize_nodes.cpp +++ b/src/duckdb/src/storage/serialization/serialize_nodes.cpp @@ -25,8 +25,8 @@ #include "duckdb/planner/table_filter.hpp" #include "duckdb/common/multi_file_reader_options.hpp" #include "duckdb/common/multi_file_reader.hpp" -#include "duckdb/execution/operator/scan/csv/csv_option.hpp" -#include "duckdb/execution/operator/scan/csv/csv_reader_options.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_option.hpp" +#include "duckdb/execution/operator/csv_scanner/csv_reader_options.hpp" #include "duckdb/function/scalar/strftime_format.hpp" #include "duckdb/function/table/read_csv.hpp" #include "duckdb/common/types/interval.hpp" @@ -128,10 +128,11 @@ void CSVReaderOptions::Serialize(Serializer &serializer) const { serializer.WriteProperty>(123, "dialect_options.state_machine_options.escape", dialect_options.state_machine_options.escape); serializer.WriteProperty>(124, "dialect_options.header", dialect_options.header); serializer.WritePropertyWithDefault(125, "dialect_options.num_cols", dialect_options.num_cols); - serializer.WriteProperty>(126, "dialect_options.new_line", dialect_options.new_line); + serializer.WriteProperty>(126, "dialect_options.state_machine_options.new_line", dialect_options.state_machine_options.new_line); serializer.WriteProperty>(127, "dialect_options.skip_rows", dialect_options.skip_rows); serializer.WriteProperty>>(128, "dialect_options.date_format", dialect_options.date_format); serializer.WritePropertyWithDefault(129, "sniffer_user_mismatch_error", sniffer_user_mismatch_error); + serializer.WritePropertyWithDefault(130, "parallel", parallel); } CSVReaderOptions CSVReaderOptions::Deserialize(Deserializer &deserializer) { @@ -162,10 +163,11 @@ CSVReaderOptions CSVReaderOptions::Deserialize(Deserializer &deserializer) { deserializer.ReadProperty>(123, "dialect_options.state_machine_options.escape", result.dialect_options.state_machine_options.escape); deserializer.ReadProperty>(124, "dialect_options.header", result.dialect_options.header); deserializer.ReadPropertyWithDefault(125, "dialect_options.num_cols", result.dialect_options.num_cols); - deserializer.ReadProperty>(126, "dialect_options.new_line", result.dialect_options.new_line); + deserializer.ReadProperty>(126, "dialect_options.state_machine_options.new_line", result.dialect_options.state_machine_options.new_line); deserializer.ReadProperty>(127, "dialect_options.skip_rows", result.dialect_options.skip_rows); deserializer.ReadProperty>>(128, "dialect_options.date_format", result.dialect_options.date_format); deserializer.ReadPropertyWithDefault(129, "sniffer_user_mismatch_error", result.sniffer_user_mismatch_error); + deserializer.ReadPropertyWithDefault(130, "parallel", result.parallel); return result; } @@ -199,6 +201,7 @@ void ColumnDefinition::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault>(102, "expression", expression); serializer.WriteProperty(103, "category", category); serializer.WriteProperty(104, "compression_type", compression_type); + serializer.WritePropertyWithDefault(105, "comment", comment, Value()); } ColumnDefinition ColumnDefinition::Deserialize(Deserializer &deserializer) { @@ -208,6 +211,7 @@ ColumnDefinition ColumnDefinition::Deserialize(Deserializer &deserializer) { auto category = deserializer.ReadProperty(103, "category"); ColumnDefinition result(std::move(name), std::move(type), std::move(expression), category); deserializer.ReadProperty(104, "compression_type", result.compression_type); + deserializer.ReadPropertyWithDefault(105, "comment", result.comment, Value()); return result; } @@ -379,9 +383,8 @@ void ReadCSVData::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault>(104, "return_names", return_names); serializer.WritePropertyWithDefault(105, "filename_col_idx", filename_col_idx); serializer.WriteProperty(106, "options", options); - serializer.WritePropertyWithDefault(107, "single_threaded", single_threaded); - serializer.WriteProperty(108, "reader_bind", reader_bind); - serializer.WritePropertyWithDefault>(109, "column_info", column_info); + serializer.WriteProperty(107, "reader_bind", reader_bind); + serializer.WritePropertyWithDefault>(108, "column_info", column_info); } unique_ptr ReadCSVData::Deserialize(Deserializer &deserializer) { @@ -393,9 +396,8 @@ unique_ptr ReadCSVData::Deserialize(Deserializer &deserializer) { deserializer.ReadPropertyWithDefault>(104, "return_names", result->return_names); deserializer.ReadPropertyWithDefault(105, "filename_col_idx", result->filename_col_idx); deserializer.ReadProperty(106, "options", result->options); - deserializer.ReadPropertyWithDefault(107, "single_threaded", result->single_threaded); - deserializer.ReadProperty(108, "reader_bind", result->reader_bind); - deserializer.ReadPropertyWithDefault>(109, "column_info", result->column_info); + deserializer.ReadProperty(107, "reader_bind", result->reader_bind); + deserializer.ReadPropertyWithDefault>(108, "column_info", result->column_info); return result; } diff --git a/src/duckdb/src/storage/serialization/serialize_parse_info.cpp b/src/duckdb/src/storage/serialization/serialize_parse_info.cpp index 149c378fc..2e0d51009 100644 --- a/src/duckdb/src/storage/serialization/serialize_parse_info.cpp +++ b/src/duckdb/src/storage/serialization/serialize_parse_info.cpp @@ -16,6 +16,7 @@ #include "duckdb/parser/parsed_data/pragma_info.hpp" #include "duckdb/parser/parsed_data/transaction_info.hpp" #include "duckdb/parser/parsed_data/vacuum_info.hpp" +#include "duckdb/parser/parsed_data/comment_on_info.hpp" namespace duckdb { @@ -33,6 +34,9 @@ unique_ptr ParseInfo::Deserialize(Deserializer &deserializer) { case ParseInfoType::ATTACH_INFO: result = AttachInfo::Deserialize(deserializer); break; + case ParseInfoType::COMMENT_ON_INFO: + result = CommentOnInfo::Deserialize(deserializer); + break; case ParseInfoType::COPY_INFO: result = CopyInfo::Deserialize(deserializer); break; @@ -85,6 +89,9 @@ unique_ptr AlterInfo::Deserialize(Deserializer &deserializer) { case AlterType::ALTER_VIEW: result = AlterViewInfo::Deserialize(deserializer); break; + case AlterType::SET_COMMENT: + result = SetCommentInfo::Deserialize(deserializer); + break; default: throw SerializationException("Unsupported type for deserialization of AlterInfo!"); } @@ -126,6 +133,9 @@ unique_ptr AlterTableInfo::Deserialize(Deserializer &deserializer) { case AlterTableType::RENAME_TABLE: result = RenameTableInfo::Deserialize(deserializer); break; + case AlterTableType::SET_COLUMN_COMMENT: + result = SetColumnCommentInfo::Deserialize(deserializer); + break; case AlterTableType::SET_DEFAULT: result = SetDefaultInfo::Deserialize(deserializer); break; @@ -222,6 +232,25 @@ unique_ptr ChangeColumnTypeInfo::Deserialize(Deserializer &deser return std::move(result); } +void CommentOnInfo::Serialize(Serializer &serializer) const { + ParseInfo::Serialize(serializer); + serializer.WriteProperty(200, "type", type); + serializer.WritePropertyWithDefault(201, "catalog", catalog); + serializer.WritePropertyWithDefault(202, "schema", schema); + serializer.WritePropertyWithDefault(203, "name", name); + serializer.WriteProperty(204, "comment", comment); +} + +unique_ptr CommentOnInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::unique_ptr(new CommentOnInfo()); + deserializer.ReadProperty(200, "type", result->type); + deserializer.ReadPropertyWithDefault(201, "catalog", result->catalog); + deserializer.ReadPropertyWithDefault(202, "schema", result->schema); + deserializer.ReadPropertyWithDefault(203, "name", result->name); + deserializer.ReadProperty(204, "comment", result->comment); + return std::move(result); +} + void CopyInfo::Serialize(Serializer &serializer) const { ParseInfo::Serialize(serializer); serializer.WritePropertyWithDefault(200, "catalog", catalog); @@ -269,6 +298,7 @@ void DropInfo::Serialize(Serializer &serializer) const { serializer.WriteProperty(204, "if_not_found", if_not_found); serializer.WritePropertyWithDefault(205, "cascade", cascade); serializer.WritePropertyWithDefault(206, "allow_drop_internal", allow_drop_internal); + serializer.WritePropertyWithDefault>(207, "extra_drop_info", extra_drop_info); } unique_ptr DropInfo::Deserialize(Deserializer &deserializer) { @@ -280,6 +310,7 @@ unique_ptr DropInfo::Deserialize(Deserializer &deserializer) { deserializer.ReadProperty(204, "if_not_found", result->if_not_found); deserializer.ReadPropertyWithDefault(205, "cascade", result->cascade); deserializer.ReadPropertyWithDefault(206, "allow_drop_internal", result->allow_drop_internal); + deserializer.ReadPropertyWithDefault>(207, "extra_drop_info", result->extra_drop_info); return std::move(result); } @@ -374,6 +405,32 @@ unique_ptr RenameViewInfo::Deserialize(Deserializer &deserializer return std::move(result); } +void SetColumnCommentInfo::Serialize(Serializer &serializer) const { + AlterTableInfo::Serialize(serializer); + serializer.WritePropertyWithDefault(400, "column_name", column_name); + serializer.WriteProperty(401, "comment", comment); +} + +unique_ptr SetColumnCommentInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::unique_ptr(new SetColumnCommentInfo()); + deserializer.ReadPropertyWithDefault(400, "column_name", result->column_name); + deserializer.ReadProperty(401, "comment", result->comment); + return std::move(result); +} + +void SetCommentInfo::Serialize(Serializer &serializer) const { + AlterInfo::Serialize(serializer); + serializer.WriteProperty(300, "entry_catalog_type", entry_catalog_type); + serializer.WriteProperty(301, "comment_value", comment_value); +} + +unique_ptr SetCommentInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::unique_ptr(new SetCommentInfo()); + deserializer.ReadProperty(300, "entry_catalog_type", result->entry_catalog_type); + deserializer.ReadProperty(301, "comment_value", result->comment_value); + return std::move(result); +} + void SetDefaultInfo::Serialize(Serializer &serializer) const { AlterTableInfo::Serialize(serializer); serializer.WritePropertyWithDefault(400, "column_name", column_name); diff --git a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp index aee88ffd2..9c7752f63 100644 --- a/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp +++ b/src/duckdb/src/storage/serialization/serialize_parsed_expression.cpp @@ -13,12 +13,14 @@ void ParsedExpression::Serialize(Serializer &serializer) const { serializer.WriteProperty(100, "class", expression_class); serializer.WriteProperty(101, "type", type); serializer.WritePropertyWithDefault(102, "alias", alias); + serializer.WritePropertyWithDefault(103, "query_location", query_location, optional_idx()); } unique_ptr ParsedExpression::Deserialize(Deserializer &deserializer) { auto expression_class = deserializer.ReadProperty(100, "class"); auto type = deserializer.ReadProperty(101, "type"); auto alias = deserializer.ReadPropertyWithDefault(102, "alias"); + auto query_location = deserializer.ReadPropertyWithDefault(103, "query_location", optional_idx()); deserializer.Set(type); unique_ptr result; switch (expression_class) { @@ -81,6 +83,7 @@ unique_ptr ParsedExpression::Deserialize(Deserializer &deseria } deserializer.Unset(); result->alias = std::move(alias); + result->query_location = query_location; return result; } @@ -335,6 +338,7 @@ void WindowExpression::Serialize(Serializer &serializer) const { serializer.WritePropertyWithDefault(212, "ignore_nulls", ignore_nulls); serializer.WritePropertyWithDefault>(213, "filter_expr", filter_expr); serializer.WritePropertyWithDefault(214, "exclude_clause", exclude_clause, WindowExcludeMode::NO_OTHER); + serializer.WritePropertyWithDefault(215, "distinct", distinct); } unique_ptr WindowExpression::Deserialize(Deserializer &deserializer) { @@ -354,6 +358,7 @@ unique_ptr WindowExpression::Deserialize(Deserializer &deseria 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); + deserializer.ReadPropertyWithDefault(215, "distinct", result->distinct); return std::move(result); } diff --git a/src/duckdb/src/storage/serialization/serialize_table_filter.cpp b/src/duckdb/src/storage/serialization/serialize_table_filter.cpp index ae065cdde..550905eef 100644 --- a/src/duckdb/src/storage/serialization/serialize_table_filter.cpp +++ b/src/duckdb/src/storage/serialization/serialize_table_filter.cpp @@ -9,6 +9,7 @@ #include "duckdb/planner/filter/null_filter.hpp" #include "duckdb/planner/filter/constant_filter.hpp" #include "duckdb/planner/filter/conjunction_filter.hpp" +#include "duckdb/planner/filter/struct_filter.hpp" namespace duckdb { @@ -35,6 +36,9 @@ unique_ptr TableFilter::Deserialize(Deserializer &deserializer) { case TableFilterType::IS_NULL: result = IsNullFilter::Deserialize(deserializer); break; + case TableFilterType::STRUCT_EXTRACT: + result = StructFilter::Deserialize(deserializer); + break; default: throw SerializationException("Unsupported type for deserialization of TableFilter!"); } @@ -94,4 +98,19 @@ unique_ptr IsNullFilter::Deserialize(Deserializer &deserializer) { return std::move(result); } +void StructFilter::Serialize(Serializer &serializer) const { + TableFilter::Serialize(serializer); + serializer.WritePropertyWithDefault(200, "child_idx", child_idx); + serializer.WritePropertyWithDefault(201, "child_name", child_name); + serializer.WritePropertyWithDefault>(202, "child_filter", child_filter); +} + +unique_ptr StructFilter::Deserialize(Deserializer &deserializer) { + auto child_idx = deserializer.ReadPropertyWithDefault(200, "child_idx"); + auto child_name = deserializer.ReadPropertyWithDefault(201, "child_name"); + auto child_filter = deserializer.ReadPropertyWithDefault>(202, "child_filter"); + auto result = duckdb::unique_ptr(new StructFilter(child_idx, std::move(child_name), std::move(child_filter))); + return std::move(result); +} + } // namespace duckdb diff --git a/src/duckdb/src/storage/serialization/serialize_tableref.cpp b/src/duckdb/src/storage/serialization/serialize_tableref.cpp index 754457e57..e97e82329 100644 --- a/src/duckdb/src/storage/serialization/serialize_tableref.cpp +++ b/src/duckdb/src/storage/serialization/serialize_tableref.cpp @@ -13,18 +13,20 @@ void TableRef::Serialize(Serializer &serializer) const { serializer.WriteProperty(100, "type", type); serializer.WritePropertyWithDefault(101, "alias", alias); serializer.WritePropertyWithDefault>(102, "sample", sample); + serializer.WritePropertyWithDefault(103, "query_location", query_location, optional_idx()); } unique_ptr TableRef::Deserialize(Deserializer &deserializer) { auto type = deserializer.ReadProperty(100, "type"); auto alias = deserializer.ReadPropertyWithDefault(101, "alias"); auto sample = deserializer.ReadPropertyWithDefault>(102, "sample"); + auto query_location = deserializer.ReadPropertyWithDefault(103, "query_location", optional_idx()); unique_ptr result; switch (type) { case TableReferenceType::BASE_TABLE: result = BaseTableRef::Deserialize(deserializer); break; - case TableReferenceType::EMPTY: + case TableReferenceType::EMPTY_FROM: result = EmptyTableRef::Deserialize(deserializer); break; case TableReferenceType::EXPRESSION_LIST: @@ -36,6 +38,9 @@ unique_ptr TableRef::Deserialize(Deserializer &deserializer) { case TableReferenceType::PIVOT: result = PivotRef::Deserialize(deserializer); break; + case TableReferenceType::SHOW_REF: + result = ShowRef::Deserialize(deserializer); + break; case TableReferenceType::SUBQUERY: result = SubqueryRef::Deserialize(deserializer); break; @@ -47,6 +52,7 @@ unique_ptr TableRef::Deserialize(Deserializer &deserializer) { } result->alias = std::move(alias); result->sample = std::move(sample); + result->query_location = query_location; return result; } @@ -135,6 +141,21 @@ unique_ptr PivotRef::Deserialize(Deserializer &deserializer) { return std::move(result); } +void ShowRef::Serialize(Serializer &serializer) const { + TableRef::Serialize(serializer); + serializer.WritePropertyWithDefault(200, "table_name", table_name); + serializer.WritePropertyWithDefault>(201, "query", query); + serializer.WriteProperty(202, "show_type", show_type); +} + +unique_ptr ShowRef::Deserialize(Deserializer &deserializer) { + auto result = duckdb::unique_ptr(new ShowRef()); + deserializer.ReadPropertyWithDefault(200, "table_name", result->table_name); + deserializer.ReadPropertyWithDefault>(201, "query", result->query); + deserializer.ReadProperty(202, "show_type", result->show_type); + return std::move(result); +} + void SubqueryRef::Serialize(Serializer &serializer) const { TableRef::Serialize(serializer); serializer.WritePropertyWithDefault>(200, "subquery", subquery); diff --git a/src/duckdb/src/storage/serialization/serialize_types.cpp b/src/duckdb/src/storage/serialization/serialize_types.cpp index c00c9b62d..0b75f518b 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::ANY_TYPE_INFO: + result = AnyTypeInfo::Deserialize(deserializer); + break; case ExtraTypeInfoType::ARRAY_TYPE_INFO: result = ArrayTypeInfo::Deserialize(deserializer); break; @@ -34,6 +37,9 @@ shared_ptr ExtraTypeInfo::Deserialize(Deserializer &deserializer) case ExtraTypeInfoType::GENERIC_TYPE_INFO: result = make_shared(type); break; + case ExtraTypeInfoType::INTEGER_LITERAL_TYPE_INFO: + result = IntegerLiteralTypeInfo::Deserialize(deserializer); + break; case ExtraTypeInfoType::INVALID_TYPE_INFO: return nullptr; case ExtraTypeInfoType::LIST_TYPE_INFO: @@ -70,6 +76,19 @@ shared_ptr AggregateStateTypeInfo::Deserialize(Deserializer &dese return std::move(result); } +void AnyTypeInfo::Serialize(Serializer &serializer) const { + ExtraTypeInfo::Serialize(serializer); + serializer.WriteProperty(200, "target_type", target_type); + serializer.WritePropertyWithDefault(201, "cast_score", cast_score); +} + +shared_ptr AnyTypeInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::shared_ptr(new AnyTypeInfo()); + deserializer.ReadProperty(200, "target_type", result->target_type); + deserializer.ReadPropertyWithDefault(201, "cast_score", result->cast_score); + return std::move(result); +} + void ArrayTypeInfo::Serialize(Serializer &serializer) const { ExtraTypeInfo::Serialize(serializer); serializer.WriteProperty(200, "child_type", child_type); @@ -96,6 +115,17 @@ shared_ptr DecimalTypeInfo::Deserialize(Deserializer &deserialize return std::move(result); } +void IntegerLiteralTypeInfo::Serialize(Serializer &serializer) const { + ExtraTypeInfo::Serialize(serializer); + serializer.WriteProperty(200, "constant_value", constant_value); +} + +shared_ptr IntegerLiteralTypeInfo::Deserialize(Deserializer &deserializer) { + auto result = duckdb::shared_ptr(new IntegerLiteralTypeInfo()); + deserializer.ReadProperty(200, "constant_value", result->constant_value); + return std::move(result); +} + void ListTypeInfo::Serialize(Serializer &serializer) const { ExtraTypeInfo::Serialize(serializer); serializer.WriteProperty(200, "child_type", child_type); diff --git a/src/duckdb/src/storage/single_file_block_manager.cpp b/src/duckdb/src/storage/single_file_block_manager.cpp index 420710c3c..322db7ee1 100644 --- a/src/duckdb/src/storage/single_file_block_manager.cpp +++ b/src/duckdb/src/storage/single_file_block_manager.cpp @@ -95,6 +95,8 @@ void DatabaseHeader::Write(WriteStream &ser) { ser.Write(meta_block); ser.Write(free_list); ser.Write(block_count); + ser.Write(block_size); + ser.Write(vector_size); } DatabaseHeader DatabaseHeader::Read(ReadStream &source) { @@ -103,6 +105,29 @@ DatabaseHeader DatabaseHeader::Read(ReadStream &source) { header.meta_block = source.Read(); header.free_list = source.Read(); header.block_count = source.Read(); + + header.block_size = source.Read(); + if (!header.block_size) { + // backwards compatibility + header.block_size = DEFAULT_BLOCK_ALLOC_SIZE; + } + if (header.block_size != Storage::BLOCK_ALLOC_SIZE) { + throw IOException("Cannot read database file: DuckDB's compiled block size is %llu bytes, but the file has a " + "block size of %llu bytes.", + Storage::BLOCK_ALLOC_SIZE, header.block_size); + } + + header.vector_size = source.Read(); + if (!header.vector_size) { + // backwards compatibility + header.vector_size = DEFAULT_STANDARD_VECTOR_SIZE; + } + if (header.vector_size != STANDARD_VECTOR_SIZE) { + throw IOException("Cannot read database file: DuckDB's compiled vector size is %llu bytes, but the file has a " + "vector size of %llu bytes.", + STANDARD_VECTOR_SIZE, header.vector_size); + } + return header; } @@ -167,19 +192,24 @@ void SingleFileBlockManager::CreateNewDatabase() { // write the database headers // initialize meta_block and free_list to INVALID_BLOCK because the database file does not contain any actual // content yet - DatabaseHeader h1, h2; + DatabaseHeader h1; // header 1 h1.iteration = 0; h1.meta_block = INVALID_BLOCK; h1.free_list = INVALID_BLOCK; h1.block_count = 0; + h1.block_size = Storage::BLOCK_ALLOC_SIZE; + h1.vector_size = STANDARD_VECTOR_SIZE; SerializeHeaderStructure(h1, header_buffer.buffer); ChecksumAndWrite(header_buffer, Storage::FILE_HEADER_SIZE); // header 2 + DatabaseHeader h2; h2.iteration = 0; h2.meta_block = INVALID_BLOCK; h2.free_list = INVALID_BLOCK; h2.block_count = 0; + h2.block_size = Storage::BLOCK_ALLOC_SIZE; + h2.vector_size = STANDARD_VECTOR_SIZE; SerializeHeaderStructure(h2, header_buffer.buffer); ChecksumAndWrite(header_buffer, Storage::FILE_HEADER_SIZE * 2ULL); // ensure that writing to disk is completed before returning @@ -205,9 +235,10 @@ void SingleFileBlockManager::LoadExistingDatabase() { DeserializeHeaderStructure(header_buffer.buffer); // read the database headers from disk - DatabaseHeader h1, h2; + DatabaseHeader h1; ReadAndChecksum(header_buffer, Storage::FILE_HEADER_SIZE); h1 = DeserializeHeaderStructure(header_buffer.buffer); + DatabaseHeader h2; ReadAndChecksum(header_buffer, Storage::FILE_HEADER_SIZE * 2ULL); h2 = DeserializeHeaderStructure(header_buffer.buffer); // check the header with the highest iteration count @@ -226,9 +257,11 @@ void SingleFileBlockManager::LoadExistingDatabase() { void SingleFileBlockManager::ReadAndChecksum(FileBuffer &block, uint64_t location) const { // read the buffer from disk block.Read(*handle, location); + // compute the checksum auto stored_checksum = Load(block.InternalBuffer()); uint64_t computed_checksum = Checksum(block.buffer, block.size); + // verify the checksum if (stored_checksum != computed_checksum) { throw IOException("Corrupt database file: computed checksum %llu does not match stored checksum %llu in block", diff --git a/src/duckdb/src/storage/standard_buffer_manager.cpp b/src/duckdb/src/storage/standard_buffer_manager.cpp index 87505a80d..df8c37d73 100644 --- a/src/duckdb/src/storage/standard_buffer_manager.cpp +++ b/src/duckdb/src/storage/standard_buffer_manager.cpp @@ -8,6 +8,7 @@ #include "duckdb/storage/buffer/buffer_pool.hpp" #include "duckdb/storage/in_memory_block_manager.hpp" #include "duckdb/storage/storage_manager.hpp" +#include "duckdb/storage/temporary_memory_manager.hpp" namespace duckdb { @@ -66,10 +67,14 @@ StandardBufferManager::StandardBufferManager(DatabaseInstance &db, string tmp) StandardBufferManager::~StandardBufferManager() { } -BufferPool &StandardBufferManager::GetBufferPool() { +BufferPool &StandardBufferManager::GetBufferPool() const { return buffer_pool; } +TemporaryMemoryManager &StandardBufferManager::GetTemporaryMemoryManager() { + return buffer_pool.GetTemporaryMemoryManager(); +} + idx_t StandardBufferManager::GetUsedMemory() const { return buffer_pool.GetUsedMemory(); } @@ -608,7 +613,7 @@ string StandardBufferManager::GetTemporaryPath(block_id_t id) { void StandardBufferManager::RequireTemporaryDirectory() { if (temp_directory.empty()) { - throw Exception( + throw InvalidInputException( "Out-of-memory: cannot write buffer because no temporary directory is specified!\nTo enable " "temporary buffer eviction set a temporary directory using PRAGMA temp_directory='/path/to/tmp.tmp'"); } diff --git a/src/duckdb/src/storage/statistics/base_statistics.cpp b/src/duckdb/src/storage/statistics/base_statistics.cpp index 9d6cdabb0..53bd862f0 100644 --- a/src/duckdb/src/storage/statistics/base_statistics.cpp +++ b/src/duckdb/src/storage/statistics/base_statistics.cpp @@ -73,6 +73,7 @@ StatisticsType BaseStatistics::GetStatsType(const LogicalType &type) { case PhysicalType::UINT32: case PhysicalType::UINT64: case PhysicalType::INT128: + case PhysicalType::UINT128: case PhysicalType::FLOAT: case PhysicalType::DOUBLE: return StatisticsType::NUMERIC_STATS; diff --git a/src/duckdb/src/storage/statistics/column_statistics.cpp b/src/duckdb/src/storage/statistics/column_statistics.cpp index adf2c3b47..e2c2b45b9 100644 --- a/src/duckdb/src/storage/statistics/column_statistics.cpp +++ b/src/duckdb/src/storage/statistics/column_statistics.cpp @@ -20,6 +20,7 @@ shared_ptr ColumnStatistics::CreateEmptyStats(const LogicalTyp void ColumnStatistics::Merge(ColumnStatistics &other) { stats.Merge(other.stats); if (distinct_stats) { + D_ASSERT(other.distinct_stats); distinct_stats->Merge(*other.distinct_stats); } } diff --git a/src/duckdb/src/storage/statistics/numeric_stats.cpp b/src/duckdb/src/storage/statistics/numeric_stats.cpp index 79e3376b2..988ecb068 100644 --- a/src/duckdb/src/storage/statistics/numeric_stats.cpp +++ b/src/duckdb/src/storage/statistics/numeric_stats.cpp @@ -98,6 +98,11 @@ hugeint_t GetNumericValueUnion::Operation(const NumericValueUnion &v) { return v.value_.hugeint; } +template <> +uhugeint_t GetNumericValueUnion::Operation(const NumericValueUnion &v) { + return v.value_.uhugeint; +} + template <> uint8_t GetNumericValueUnion::Operation(const NumericValueUnion &v) { return v.value_.utinyint; @@ -248,6 +253,8 @@ FilterPropagateResult NumericStats::CheckZonemap(const BaseStatistics &stats, Ex return CheckZonemapTemplated(stats, comparison_type, constant); case PhysicalType::INT128: return CheckZonemapTemplated(stats, comparison_type, constant); + case PhysicalType::UINT128: + return CheckZonemapTemplated(stats, comparison_type, constant); case PhysicalType::FLOAT: return CheckZonemapTemplated(stats, comparison_type, constant); case PhysicalType::DOUBLE: @@ -301,6 +308,9 @@ void SetNumericValueInternal(const Value &input, const LogicalType &type, Numeri case PhysicalType::INT128: val.value_.hugeint = HugeIntValue::Get(input); break; + case PhysicalType::UINT128: + val.value_.uhugeint = UhugeIntValue::Get(input); + break; case PhysicalType::FLOAT: val.value_.float_ = FloatValue::Get(input); break; @@ -344,6 +354,8 @@ Value NumericValueUnionToValueInternal(const LogicalType &type, const NumericVal return Value::UBIGINT(val.value_.ubigint); case PhysicalType::INT128: return Value::HUGEINT(val.value_.hugeint); + case PhysicalType::UINT128: + return Value::UHUGEINT(val.value_.uhugeint); case PhysicalType::FLOAT: return Value::FLOAT(val.value_.float_); case PhysicalType::DOUBLE: @@ -442,6 +454,9 @@ static void SerializeNumericStatsValue(const LogicalType &type, NumericValueUnio case PhysicalType::INT128: serializer.WriteProperty(101, "value", val.value_.hugeint); break; + case PhysicalType::UINT128: + serializer.WriteProperty(101, "value", val.value_.uhugeint); + break; case PhysicalType::FLOAT: serializer.WriteProperty(101, "value", val.value_.float_); break; @@ -492,6 +507,9 @@ static void DeserializeNumericStatsValue(const LogicalType &type, NumericValueUn case PhysicalType::INT128: result.value_.hugeint = deserializer.ReadProperty(101, "value"); break; + case PhysicalType::UINT128: + result.value_.uhugeint = deserializer.ReadProperty(101, "value"); + break; case PhysicalType::FLOAT: result.value_.float_ = deserializer.ReadProperty(101, "value"); break; @@ -587,6 +605,9 @@ void NumericStats::Verify(const BaseStatistics &stats, Vector &vector, const Sel case PhysicalType::INT128: TemplatedVerify(stats, vector, sel, count); break; + case PhysicalType::UINT128: + TemplatedVerify(stats, vector, sel, count); + break; case PhysicalType::FLOAT: TemplatedVerify(stats, vector, sel, count); break; diff --git a/src/duckdb/src/storage/statistics/numeric_stats_union.cpp b/src/duckdb/src/storage/statistics/numeric_stats_union.cpp index ab6bc6739..15aaa11b3 100644 --- a/src/duckdb/src/storage/statistics/numeric_stats_union.cpp +++ b/src/duckdb/src/storage/statistics/numeric_stats_union.cpp @@ -32,6 +32,11 @@ hugeint_t &NumericValueUnion::GetReferenceUnsafe() { return value_.hugeint; } +template <> +uhugeint_t &NumericValueUnion::GetReferenceUnsafe() { + return value_.uhugeint; +} + template <> uint8_t &NumericValueUnion::GetReferenceUnsafe() { return value_.utinyint; diff --git a/src/duckdb/src/storage/statistics/string_stats.cpp b/src/duckdb/src/storage/statistics/string_stats.cpp index c9be1c386..9b571fa81 100644 --- a/src/duckdb/src/storage/statistics/string_stats.cpp +++ b/src/duckdb/src/storage/statistics/string_stats.cpp @@ -159,8 +159,8 @@ void StringStats::Update(BaseStatistics &stats, const string_t &value) { if (unicode == UnicodeType::UNICODE) { string_data.has_unicode = true; } else if (unicode == UnicodeType::INVALID) { - throw InvalidInputException(ErrorManager::InvalidUnicodeError(string(const_char_ptr_cast(data), size), - "segment statistics update")); + throw ErrorManager::InvalidUnicodeError(string(const_char_ptr_cast(data), size), + "segment statistics update"); } } } diff --git a/src/duckdb/src/storage/storage_manager.cpp b/src/duckdb/src/storage/storage_manager.cpp index 8131c4652..832924afe 100644 --- a/src/duckdb/src/storage/storage_manager.cpp +++ b/src/duckdb/src/storage/storage_manager.cpp @@ -52,6 +52,32 @@ bool ObjectCache::ObjectCacheEnabled(ClientContext &context) { return context.db->config.options.object_cache_enable; } +optional_ptr StorageManager::GetWriteAheadLog() { + if (InMemory() || read_only || !load_complete) { + return nullptr; + } + + if (wal) { + return wal.get(); + } + + // lazy WAL creation + wal = make_uniq(db, GetWALPath()); + return wal.get(); +} + +string StorageManager::GetWALPath() { + + std::size_t question_mark_pos = path.find('?'); + auto wal_path = path; + if (question_mark_pos != std::string::npos) { + wal_path.insert(question_mark_pos, ".wal"); + } else { + wal_path += ".wal"; + } + return wal_path; +} + bool StorageManager::InMemory() { D_ASSERT(!path.empty()); return path == IN_MEMORY_PATH; @@ -92,21 +118,15 @@ SingleFileStorageManager::SingleFileStorageManager(AttachedDatabase &db, string } void SingleFileStorageManager::LoadDatabase() { + if (InMemory()) { block_manager = make_uniq(BufferManager::GetBufferManager(db)); table_io_manager = make_uniq(*block_manager); return; } - std::size_t question_mark_pos = path.find('?'); - auto wal_path = path; - if (question_mark_pos != std::string::npos) { - wal_path.insert(question_mark_pos, ".wal"); - } else { - wal_path += ".wal"; - } + auto &fs = FileSystem::Get(db); auto &config = DBConfig::Get(db); - bool truncate_wal = false; if (!config.options.enable_external_access) { if (!db.IsInitialDatabase()) { throw PermissionException("Attaching on-disk databases is disabled through configuration"); @@ -117,22 +137,27 @@ void SingleFileStorageManager::LoadDatabase() { options.read_only = read_only; options.use_direct_io = config.options.use_direct_io; options.debug_initialize = config.options.debug_initialize; + // first check if the database exists if (!fs.FileExists(path)) { if (read_only) { throw CatalogException("Cannot open database \"%s\" in read-only mode: database does not exist", path); } + // check if the WAL exists + auto wal_path = GetWALPath(); if (fs.FileExists(wal_path)) { // WAL file exists but database file does not // remove the WAL fs.RemoveFile(wal_path); } + // initialize the block manager while creating a new db file auto sf_block_manager = make_uniq(db, path, options); sf_block_manager->CreateNewDatabase(); block_manager = std::move(sf_block_manager); table_io_manager = make_uniq(*block_manager); + } else { // initialize the block manager while loading the current db file auto sf_block_manager = make_uniq(db, path, options); @@ -140,22 +165,21 @@ void SingleFileStorageManager::LoadDatabase() { block_manager = std::move(sf_block_manager); table_io_manager = make_uniq(*block_manager); - //! Load from storage - auto checkpointer = SingleFileCheckpointReader(*this); - checkpointer.LoadFromStorage(); + // load the db from storage + auto checkpoint_reader = SingleFileCheckpointReader(*this); + checkpoint_reader.LoadFromStorage(); + // check if the WAL file exists + auto wal_path = GetWALPath(); if (fs.FileExists(wal_path)) { // replay the WAL - truncate_wal = WriteAheadLog::Replay(db, wal_path); - } - } - // initialize the WAL file - if (!read_only) { - wal = make_uniq(db, wal_path); - if (truncate_wal) { - wal->Truncate(0); + if (WriteAheadLog::Replay(db, wal_path)) { + fs.RemoveFile(wal_path); + } } } + + load_complete = true; } /////////////////////////////////////////////////////////////////////////////// @@ -239,7 +263,8 @@ void SingleFileStorageManager::CreateCheckpoint(bool delete_wal, bool force_chec SingleFileCheckpointWriter checkpointer(db, *block_manager); checkpointer.CreateCheckpoint(); } catch (std::exception &ex) { - throw FatalException("Failed to create checkpoint because of error: %s", ex.what()); + ErrorData error(ex); + throw FatalException("Failed to create checkpoint because of error: %s", error.RawMessage()); } } if (delete_wal) { diff --git a/src/duckdb/src/storage/table/array_column_data.cpp b/src/duckdb/src/storage/table/array_column_data.cpp index 993e3cdd0..31cb93fac 100644 --- a/src/duckdb/src/storage/table/array_column_data.cpp +++ b/src/duckdb/src/storage/table/array_column_data.cpp @@ -146,16 +146,26 @@ unique_ptr ArrayColumnData::GetUpdateStatistics() { 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()); + // Create state for validity & child column + if (state.child_states.empty()) { + 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 &child_type = ArrayType::GetChildType(type); auto array_size = ArrayType::GetSize(type); - child_column->FetchRow(transaction, *state.child_states[1], row_id * array_size, child_vec, result_idx); + + // We need to fetch between [row_id * array_size, (row_id + 1) * array_size) + auto child_state = make_uniq(); + child_state->Initialize(child_type, nullptr); + child_column->InitializeScanWithOffset(*child_state, row_id * array_size); + Vector child_scan(child_type, array_size); + child_column->ScanCount(*child_state, child_scan, array_size); + VectorOperations::Copy(child_scan, child_vec, array_size, 0, result_idx * array_size); } void ArrayColumnData::CommitDropColumn() { @@ -202,10 +212,13 @@ unique_ptr ArrayColumnData::Checkpoint(RowGroup &row_grou return std::move(checkpoint_state); } -void ArrayColumnData::DeserializeColumn(Deserializer &deserializer) { - deserializer.ReadObject(101, "validity", [&](Deserializer &source) { validity.DeserializeColumn(source); }); +void ArrayColumnData::DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) { + deserializer.ReadObject(101, "validity", + [&](Deserializer &source) { validity.DeserializeColumn(source, target_stats); }); + + auto &child_stats = ArrayStats::GetChildStats(target_stats); deserializer.ReadObject(102, "child_column", - [&](Deserializer &source) { child_column->DeserializeColumn(source); }); + [&](Deserializer &source) { child_column->DeserializeColumn(source, child_stats); }); this->count = validity.count; } diff --git a/src/duckdb/src/storage/table/chunk_info.cpp b/src/duckdb/src/storage/table/chunk_info.cpp index 344b6c4d6..d220e0809 100644 --- a/src/duckdb/src/storage/table/chunk_info.cpp +++ b/src/duckdb/src/storage/table/chunk_info.cpp @@ -1,5 +1,6 @@ #include "duckdb/storage/table/chunk_info.hpp" #include "duckdb/transaction/transaction.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" #include "duckdb/common/serializer/memory_stream.hpp" diff --git a/src/duckdb/src/storage/table/column_data.cpp b/src/duckdb/src/storage/table/column_data.cpp index 96eeed1a0..ef9b0e66b 100644 --- a/src/duckdb/src/storage/table/column_data.cpp +++ b/src/duckdb/src/storage/table/column_data.cpp @@ -1,5 +1,5 @@ #include "duckdb/storage/table/column_data.hpp" - +#include "duckdb/common/exception/transaction_exception.hpp" #include "duckdb/common/vector_operations/vector_operations.hpp" #include "duckdb/function/compression_function.hpp" #include "duckdb/planner/table_filter.hpp" @@ -113,8 +113,15 @@ idx_t ColumnData::ScanVector(ColumnScanState &state, Vector &result, idx_t remai idx_t scan_count = MinValue(remaining, state.current->start + state.current->count - state.row_index); idx_t result_offset = initial_remaining - remaining; if (scan_count > 0) { - state.current->Scan(state, scan_count, result, result_offset, - !has_updates && scan_count == initial_remaining); + if (state.scan_options && state.scan_options->force_fetch_row) { + for (idx_t i = 0; i < scan_count; i++) { + ColumnFetchState fetch_state; + state.current->FetchRow(fetch_state, state.row_index + i, result, result_offset + i); + } + } else { + state.current->Scan(state, scan_count, result, result_offset, + !has_updates && scan_count == initial_remaining); + } state.row_index += scan_count; remaining -= scan_count; @@ -387,14 +394,18 @@ unique_ptr ColumnData::GetUpdateStatistics() { } void ColumnData::AppendTransientSegment(SegmentLock &l, idx_t start_row) { - idx_t segment_size = Storage::BLOCK_SIZE; + + idx_t vector_segment_size = Storage::BLOCK_SIZE; if (start_row == idx_t(MAX_ROW_ID)) { #if STANDARD_VECTOR_SIZE < 1024 - segment_size = 1024 * GetTypeIdSize(type.InternalType()); + vector_segment_size = 1024 * GetTypeIdSize(type.InternalType()); #else - segment_size = STANDARD_VECTOR_SIZE * GetTypeIdSize(type.InternalType()); + vector_segment_size = STANDARD_VECTOR_SIZE * GetTypeIdSize(type.InternalType()); #endif } + + // the segment size is bound by the block size, but can be smaller + idx_t segment_size = Storage::BLOCK_SIZE < vector_segment_size ? Storage::BLOCK_SIZE : vector_segment_size; auto new_segment = ColumnSegment::CreateTransientSegment(GetDatabase(), type, start_row, segment_size); data.AppendSegment(l, std::move(new_segment)); } @@ -413,7 +424,15 @@ unique_ptr ColumnData::CreateCheckpointState(RowGroup &ro void ColumnData::CheckpointScan(ColumnSegment &segment, ColumnScanState &state, idx_t row_group_start, idx_t count, Vector &scan_vector) { - segment.Scan(state, count, scan_vector, 0, true); + if (state.scan_options && state.scan_options->force_fetch_row) { + for (idx_t i = 0; i < count; i++) { + ColumnFetchState fetch_state; + segment.FetchRow(fetch_state, state.row_index + i, scan_vector, i); + } + } else { + segment.Scan(state, count, scan_vector, 0, true); + } + if (updates) { scan_vector.Flatten(count); updates->FetchCommittedRange(state.row_index - row_group_start, count, scan_vector); @@ -447,7 +466,7 @@ unique_ptr ColumnData::Checkpoint(RowGroup &row_group, return checkpoint_state; } -void ColumnData::DeserializeColumn(Deserializer &deserializer) { +void ColumnData::DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) { // load the data pointers for the column deserializer.Set(info.db.GetDatabase()); deserializer.Set(type); @@ -463,9 +482,11 @@ void ColumnData::DeserializeColumn(Deserializer &deserializer) { for (auto &data_pointer : data_pointers) { // Update the count and statistics this->count += data_pointer.tuple_count; - if (stats) { - stats->statistics.Merge(data_pointer.statistics); - } + + // Merge the statistics. If this is a child column, the target_stats reference will point into the parents stats + // otherwise if this is a top level column, `stats->statistics` == `target_stats` + + target_stats.Merge(data_pointer.statistics); // create a persistent segment auto segment = ColumnSegment::CreatePersistentSegment( @@ -478,12 +499,11 @@ void ColumnData::DeserializeColumn(Deserializer &deserializer) { } shared_ptr ColumnData::Deserialize(BlockManager &block_manager, DataTableInfo &info, idx_t column_index, - idx_t start_row, ReadStream &source, const LogicalType &type, - optional_ptr parent) { - auto entry = ColumnData::CreateColumn(block_manager, info, column_index, start_row, type, parent); + idx_t start_row, ReadStream &source, const LogicalType &type) { + auto entry = ColumnData::CreateColumn(block_manager, info, column_index, start_row, type, nullptr); BinaryDeserializer deserializer(source); deserializer.Begin(); - entry->DeserializeColumn(deserializer); + entry->DeserializeColumn(deserializer, entry->stats->statistics); deserializer.End(); return entry; } diff --git a/src/duckdb/src/storage/table/column_data_checkpointer.cpp b/src/duckdb/src/storage/table/column_data_checkpointer.cpp index 271c67eb6..8c9438308 100644 --- a/src/duckdb/src/storage/table/column_data_checkpointer.cpp +++ b/src/duckdb/src/storage/table/column_data_checkpointer.cpp @@ -63,8 +63,13 @@ void ColumnDataCheckpointer::ScanSegments(const std::function> &compression_functions, CompressionType compression_type) { - // On of the force_compression flags has been set - // check if this compression method is available +// On of the force_compression flags has been set +// check if this compression method is available +#ifdef DEBUG + if (CompressionTypeIsDeprecated(compression_type)) { + throw InternalException("Deprecated compression type: %s", CompressionTypeToString(compression_type)); + } +#endif bool found = false; for (idx_t i = 0; i < compression_functions.size(); i++) { auto &compression_function = *compression_functions[i]; @@ -120,7 +125,10 @@ unique_ptr ColumnDataCheckpointer::DetectBestCompressionMethod(idx if (!compression_functions[i]) { continue; } - auto success = compression_functions[i]->analyze(*analyze_states[i], scan_vector, count); + bool success = false; + if (analyze_states[i]) { + success = compression_functions[i]->analyze(*analyze_states[i], scan_vector, count); + } if (!success) { // could not use this compression function on this data set // erase it @@ -185,6 +193,7 @@ void ColumnDataCheckpointer::WriteToDisk() { // now that we have analyzed the compression functions we can start writing to disk auto best_function = compression_functions[compression_idx]; auto compress_state = best_function->init_compression(*this, std::move(analyze_state)); + ScanSegments( [&](Vector &scan_vector, idx_t count) { best_function->compress(*compress_state, scan_vector, count); }); best_function->compress_finalize(*compress_state); diff --git a/src/duckdb/src/storage/table/column_segment.cpp b/src/duckdb/src/storage/table/column_segment.cpp index b97c12291..5a77c881c 100644 --- a/src/duckdb/src/storage/table/column_segment.cpp +++ b/src/duckdb/src/storage/table/column_segment.cpp @@ -7,6 +7,7 @@ #include "duckdb/storage/storage_manager.hpp" #include "duckdb/planner/filter/conjunction_filter.hpp" #include "duckdb/planner/filter/constant_filter.hpp" +#include "duckdb/planner/filter/struct_filter.hpp" #include "duckdb/main/config.hpp" #include "duckdb/storage/table/scan_state.hpp" #include "duckdb/storage/data_pointer.hpp" @@ -15,15 +16,21 @@ namespace duckdb { +//===--------------------------------------------------------------------===// +// Create +//===--------------------------------------------------------------------===// + unique_ptr ColumnSegment::CreatePersistentSegment(DatabaseInstance &db, BlockManager &block_manager, block_id_t block_id, idx_t offset, const LogicalType &type, idx_t start, idx_t count, CompressionType compression_type, BaseStatistics statistics, unique_ptr segment_state) { + auto &config = DBConfig::GetConfig(db); optional_ptr function; shared_ptr block; + if (block_id == INVALID_BLOCK) { // constant segment, no need to allocate an actual block function = config.GetCompressionFunction(CompressionType::COMPRESSION_CONSTANT, type.InternalType()); @@ -31,6 +38,7 @@ unique_ptr ColumnSegment::CreatePersistentSegment(DatabaseInstanc function = config.GetCompressionFunction(compression_type, type.InternalType()); block = block_manager.RegisterBlock(block_id); } + auto segment_size = Storage::BLOCK_SIZE; return make_uniq(db, std::move(block), type, ColumnSegmentType::PERSISTENT, start, count, *function, std::move(statistics), block_id, offset, segment_size, std::move(segment_state)); @@ -38,10 +46,14 @@ unique_ptr ColumnSegment::CreatePersistentSegment(DatabaseInstanc unique_ptr ColumnSegment::CreateTransientSegment(DatabaseInstance &db, const LogicalType &type, idx_t start, idx_t segment_size) { + + D_ASSERT(segment_size <= Storage::BLOCK_SIZE); + auto &config = DBConfig::GetConfig(db); auto function = config.GetCompressionFunction(CompressionType::COMPRESSION_UNCOMPRESSED, type.InternalType()); auto &buffer_manager = BufferManager::GetBufferManager(db); shared_ptr block; + // transient: allocate a buffer for the uncompressed segment if (segment_size < Storage::BLOCK_SIZE) { block = buffer_manager.RegisterSmallMemory(segment_size); @@ -52,28 +64,33 @@ unique_ptr ColumnSegment::CreateTransientSegment(DatabaseInstance BaseStatistics::CreateEmpty(type), INVALID_BLOCK, 0, segment_size); } -unique_ptr ColumnSegment::CreateSegment(ColumnSegment &other, idx_t start) { - return make_uniq(other, start); -} - +//===--------------------------------------------------------------------===// +// Construct/Destruct +//===--------------------------------------------------------------------===// ColumnSegment::ColumnSegment(DatabaseInstance &db, shared_ptr block, LogicalType type_p, ColumnSegmentType segment_type, idx_t start, idx_t count, CompressionFunction &function_p, BaseStatistics statistics, block_id_t block_id_p, idx_t offset_p, idx_t segment_size_p, unique_ptr segment_state) + : SegmentBase(start, count), db(db), type(std::move(type_p)), type_size(GetTypeIdSize(type.InternalType())), segment_type(segment_type), function(function_p), stats(std::move(statistics)), block(std::move(block)), block_id(block_id_p), offset(offset_p), segment_size(segment_size_p) { + if (function.get().init_segment) { this->segment_state = function.get().init_segment(*this, block_id, segment_state.get()); } + D_ASSERT(segment_size <= Storage::BLOCK_SIZE); } ColumnSegment::ColumnSegment(ColumnSegment &other, idx_t start) + : SegmentBase(start, other.count.load()), db(other.db), type(std::move(other.type)), type_size(other.type_size), segment_type(other.segment_type), function(other.function), stats(std::move(other.stats)), block(std::move(other.block)), block_id(other.block_id), offset(other.offset), segment_size(other.segment_size), segment_state(std::move(other.segment_state)) { + + D_ASSERT(segment_size <= Storage::BLOCK_SIZE); } ColumnSegment::~ColumnSegment() { @@ -128,11 +145,14 @@ idx_t ColumnSegment::SegmentSize() const { void ColumnSegment::Resize(idx_t new_size) { D_ASSERT(new_size > this->segment_size); D_ASSERT(offset == 0); + D_ASSERT(new_size <= Storage::BLOCK_SIZE); + auto &buffer_manager = BufferManager::GetBufferManager(db); auto old_handle = buffer_manager.Pin(block); shared_ptr new_block; - auto new_handle = buffer_manager.Allocate(Storage::BLOCK_SIZE, false, &new_block); + auto new_handle = buffer_manager.Allocate(new_size, false, &new_block); memcpy(new_handle.Ptr(), old_handle.Ptr(), segment_size); + this->block_id = new_block->BlockId(); this->block = std::move(new_block); this->segment_size = new_size; @@ -439,6 +459,13 @@ idx_t ColumnSegment::FilterSelection(SelectionVector &sel, Vector &result, const constant_filter.comparison_type, mask); break; } + case PhysicalType::UINT128: { + auto result_flat = FlatVector::GetData(result); + auto predicate = UhugeIntValue::Get(constant_filter.constant); + FilterSelectionSwitch(result_flat, predicate, sel, approved_tuple_count, + constant_filter.comparison_type, mask); + break; + } case PhysicalType::FLOAT: { auto result_flat = FlatVector::GetData(result); auto predicate = FloatValue::Get(constant_filter.constant); @@ -476,6 +503,13 @@ idx_t ColumnSegment::FilterSelection(SelectionVector &sel, Vector &result, const return TemplatedNullSelection(sel, approved_tuple_count, mask); case TableFilterType::IS_NOT_NULL: return TemplatedNullSelection(sel, approved_tuple_count, mask); + case TableFilterType::STRUCT_EXTRACT: { + auto &struct_filter = filter.Cast(); + // Apply the filter on the child vector + auto &child_vec = StructVector::GetEntries(result)[struct_filter.child_idx]; + auto &child_mask = FlatVector::Validity(*child_vec); + return FilterSelection(sel, *child_vec, *struct_filter.child_filter, approved_tuple_count, child_mask); + } default: throw InternalException("FIXME: unsupported type for filter selection"); } diff --git a/src/duckdb/src/storage/table/list_column_data.cpp b/src/duckdb/src/storage/table/list_column_data.cpp index cd87ec1e5..b805497a5 100644 --- a/src/duckdb/src/storage/table/list_column_data.cpp +++ b/src/duckdb/src/storage/table/list_column_data.cpp @@ -132,14 +132,14 @@ void ListColumnData::Skip(ColumnScanState &state, idx_t count) { // we need to read the list entries/offsets to figure out how much to skip // note that we only need to read the first and last entry // however, let's just read all "count" entries for now - Vector result(LogicalType::UBIGINT, count); - idx_t scan_count = ScanVector(state, result, count, false); - if (scan_count == 0) { - return; - } + Vector offset_vector(LogicalType::UBIGINT, count); + idx_t scan_count = ScanVector(state, offset_vector, count, false); + D_ASSERT(scan_count > 0); - auto data = FlatVector::GetData(result); - auto last_entry = data[scan_count - 1]; + UnifiedVectorFormat offsets; + offset_vector.ToUnifiedFormat(scan_count, offsets); + auto data = UnifiedVectorFormat::GetData(offsets); + auto last_entry = data[offsets.sel->get_index(scan_count - 1)]; idx_t child_scan_count = last_entry - state.last_offset; if (child_scan_count == 0) { return; @@ -292,7 +292,7 @@ void ListColumnData::FetchRow(TransactionData transaction, ColumnFetchState &sta auto &child_type = ListType::GetChildType(result.GetType()); Vector child_scan(child_type, child_scan_count); // seek the scan towards the specified position and read [length] entries - child_state->Initialize(child_type); + child_state->Initialize(child_type, nullptr); child_column->InitializeScanWithOffset(*child_state, start + start_offset); D_ASSERT(child_type.InternalType() == PhysicalType::STRUCT || child_state->row_index + child_scan_count - this->start <= child_column->GetMaxEntry()); @@ -350,14 +350,16 @@ unique_ptr ListColumnData::Checkpoint(RowGroup &row_group return base_state; } -void ListColumnData::DeserializeColumn(Deserializer &deserializer) { - ColumnData::DeserializeColumn(deserializer); +void ListColumnData::DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) { + ColumnData::DeserializeColumn(deserializer, target_stats); - deserializer.ReadObject(101, "validity", - [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer); }); + deserializer.ReadObject( + 101, "validity", [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer, target_stats); }); - deserializer.ReadObject(102, "child_column", - [&](Deserializer &deserializer) { child_column->DeserializeColumn(deserializer); }); + auto &child_stats = ListStats::GetChildStats(target_stats); + deserializer.ReadObject(102, "child_column", [&](Deserializer &deserializer) { + child_column->DeserializeColumn(deserializer, child_stats); + }); } void ListColumnData::GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, diff --git a/src/duckdb/src/storage/table/row_group.cpp b/src/duckdb/src/storage/table/row_group.cpp index 5718ac627..b4572ebfb 100644 --- a/src/duckdb/src/storage/table/row_group.cpp +++ b/src/duckdb/src/storage/table/row_group.cpp @@ -20,6 +20,8 @@ #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" #include "duckdb/common/serializer/binary_serializer.hpp" +#include "duckdb/planner/filter/conjunction_filter.hpp" +#include "duckdb/planner/filter/struct_filter.hpp" namespace duckdb { @@ -99,7 +101,7 @@ ColumnData &RowGroup::GetColumn(storage_t c) { auto &block_pointer = column_pointers[c]; MetadataReader column_data_reader(metadata_manager, block_pointer); this->columns[c] = - ColumnData::Deserialize(GetBlockManager(), GetTableInfo(), c, start, column_data_reader, types[c], nullptr); + ColumnData::Deserialize(GetBlockManager(), GetTableInfo(), c, start, column_data_reader, types[c]); is_loaded[c] = true; if (this->columns[c]->count != this->count) { throw InternalException("Corrupted database - loaded column with index %llu at row start %llu, count %llu did " @@ -125,7 +127,10 @@ void RowGroup::InitializeEmpty(const vector &types) { } } -void ColumnScanState::Initialize(const LogicalType &type) { +void ColumnScanState::Initialize(const LogicalType &type, optional_ptr options) { + // Register the options in the state + scan_options = options; + if (type.id() == LogicalTypeId::VALIDITY) { // validity - nothing to initialize return; @@ -135,19 +140,23 @@ void ColumnScanState::Initialize(const LogicalType &type) { auto &struct_children = StructType::GetChildTypes(type); child_states.resize(struct_children.size() + 1); for (idx_t i = 0; i < struct_children.size(); i++) { - child_states[i + 1].Initialize(struct_children[i].second); + child_states[i + 1].Initialize(struct_children[i].second, options); } + child_states[0].scan_options = options; } else if (type.InternalType() == PhysicalType::LIST) { // validity + list child child_states.resize(2); - child_states[1].Initialize(ListType::GetChildType(type)); + child_states[1].Initialize(ListType::GetChildType(type), options); + child_states[0].scan_options = options; } else if (type.InternalType() == PhysicalType::ARRAY) { // validity + array child child_states.resize(2); - child_states[1].Initialize(ArrayType::GetChildType(type)); + child_states[0].scan_options = options; + child_states[1].Initialize(ArrayType::GetChildType(type), options); } else { // validity child_states.resize(1); + child_states[0].scan_options = options; } } @@ -158,7 +167,7 @@ void CollectionScanState::Initialize(const vector &types) { if (column_ids[i] == COLUMN_IDENTIFIER_ROW_ID) { continue; } - column_scans[i].Initialize(types[column_ids[i]]); + column_scans[i].Initialize(types[column_ids[i]], &GetOptions()); } } @@ -181,6 +190,7 @@ bool RowGroup::InitializeScanWithOffset(CollectionScanState &state, idx_t vector if (column != COLUMN_IDENTIFIER_ROW_ID) { auto &column_data = GetColumn(column); column_data.InitializeScanWithOffset(state.column_scans[i], start + vector_offset * STANDARD_VECTOR_SIZE); + state.column_scans[i].scan_options = &state.GetOptions(); } else { state.column_scans[i].current = nullptr; } @@ -209,6 +219,7 @@ bool RowGroup::InitializeScan(CollectionScanState &state) { if (column != COLUMN_IDENTIFIER_ROW_ID) { auto &column_data = GetColumn(column); column_data.InitializeScan(state.column_scans[i]); + state.column_scans[i].scan_options = &state.GetOptions(); } else { state.column_scans[i].current = nullptr; } @@ -353,6 +364,40 @@ bool RowGroup::CheckZonemap(TableFilterSet &filters, const vector &co return true; } +static idx_t GetFilterScanCount(ColumnScanState &state, TableFilter &filter) { + switch (filter.filter_type) { + case TableFilterType::STRUCT_EXTRACT: { + auto &struct_filter = filter.Cast(); + auto &child_state = state.child_states[1 + struct_filter.child_idx]; // +1 for validity + auto &child_filter = struct_filter.child_filter; + return GetFilterScanCount(child_state, *child_filter); + } + case TableFilterType::CONJUNCTION_AND: { + auto &conjunction_state = filter.Cast(); + idx_t max_count = 0; + for (auto &child_filter : conjunction_state.child_filters) { + max_count = std::max(GetFilterScanCount(state, *child_filter), max_count); + } + return max_count; + } + case TableFilterType::CONJUNCTION_OR: { + auto &conjunction_state = filter.Cast(); + idx_t max_count = 0; + for (auto &child_filter : conjunction_state.child_filters) { + max_count = std::max(GetFilterScanCount(state, *child_filter), max_count); + } + return max_count; + } + case TableFilterType::IS_NULL: + case TableFilterType::IS_NOT_NULL: + case TableFilterType::CONSTANT_COMPARISON: + return state.current->start + state.current->count; + default: { + throw NotImplementedException("Unimplemented filter type for zonemap"); + } + } +} + bool RowGroup::CheckZonemapSegments(CollectionScanState &state) { auto &column_ids = state.GetColumnIds(); auto filters = state.GetFilters(); @@ -365,8 +410,9 @@ bool RowGroup::CheckZonemapSegments(CollectionScanState &state) { const auto &base_column_idx = column_ids[column_idx]; bool read_segment = GetColumn(base_column_idx).CheckZonemap(state.column_scans[column_idx], *entry.second); if (!read_segment) { - idx_t target_row = - state.column_scans[column_idx].current->start + state.column_scans[column_idx].current->count; + + idx_t target_row = GetFilterScanCount(state.column_scans[column_idx], *entry.second); + D_ASSERT(target_row >= this->start); D_ASSERT(target_row <= this->start + this->count); idx_t target_vector_index = (target_row - this->start) / STANDARD_VECTOR_SIZE; diff --git a/src/duckdb/src/storage/table/row_group_collection.cpp b/src/duckdb/src/storage/table/row_group_collection.cpp index 2d4a00f61..a8fc2e81b 100644 --- a/src/duckdb/src/storage/table/row_group_collection.cpp +++ b/src/duckdb/src/storage/table/row_group_collection.cpp @@ -616,7 +616,7 @@ struct CollectionCheckpointState { mutex write_lock; public: - void PushError(PreservedError error) { + void PushError(ErrorData error) { error_manager.PushError(std::move(error)); } bool HasError() { @@ -667,12 +667,10 @@ class BaseCheckpointTask : public Task { ExecuteTask(); checkpoint_state.FinishTask(); return TaskExecutionResult::TASK_FINISHED; - } catch (Exception &ex) { - checkpoint_state.PushError(PreservedError(ex)); } catch (std::exception &ex) { - checkpoint_state.PushError(PreservedError(ex)); + checkpoint_state.PushError(ErrorData(ex)); } catch (...) { // LCOV_EXCL_START - checkpoint_state.PushError(PreservedError("Unknown exception during Checkpoint!")); + checkpoint_state.PushError(ErrorData("Unknown exception during Checkpoint!")); } // LCOV_EXCL_STOP return TaskExecutionResult::TASK_ERROR; } diff --git a/src/duckdb/src/storage/table/row_version_manager.cpp b/src/duckdb/src/storage/table/row_version_manager.cpp index 945d0a3b1..e1ffecb4b 100644 --- a/src/duckdb/src/storage/table/row_version_manager.cpp +++ b/src/duckdb/src/storage/table/row_version_manager.cpp @@ -219,7 +219,8 @@ shared_ptr RowVersionManager::Deserialize(MetaBlockPointer de for (idx_t i = 0; i < chunk_count; i++) { idx_t vector_index = source.Read(); if (vector_index >= Storage::ROW_GROUP_VECTOR_COUNT) { - throw Exception("In DeserializeDeletes, vector_index is out of range for the row group. Corrupted file?"); + throw InternalException( + "In DeserializeDeletes, vector_index is out of range for the row group. Corrupted file?"); } version_info->vector_info[vector_index] = ChunkInfo::Read(source); } diff --git a/src/duckdb/src/storage/table/scan_state.cpp b/src/duckdb/src/storage/table/scan_state.cpp index a40a9b1f3..6931d1056 100644 --- a/src/duckdb/src/storage/table/scan_state.cpp +++ b/src/duckdb/src/storage/table/scan_state.cpp @@ -67,6 +67,10 @@ AdaptiveFilter *CollectionScanState::GetAdaptiveFilter() { return parent.GetAdaptiveFilter(); } +TableScanOptions &CollectionScanState::GetOptions() { + return parent.options; +} + ParallelCollectionScanState::ParallelCollectionScanState() : collection(nullptr), current_row_group(nullptr), processed_rows(0) { } diff --git a/src/duckdb/src/storage/table/standard_column_data.cpp b/src/duckdb/src/storage/table/standard_column_data.cpp index de3668ac8..9347e72e3 100644 --- a/src/duckdb/src/storage/table/standard_column_data.cpp +++ b/src/duckdb/src/storage/table/standard_column_data.cpp @@ -105,6 +105,7 @@ idx_t StandardColumnData::Fetch(ColumnScanState &state, row_t row_id, Vector &re // fetch validity mask if (state.child_states.empty()) { ColumnScanState child_state; + child_state.scan_options = state.scan_options; state.child_states.push_back(std::move(child_state)); } auto scan_count = ColumnData::Fetch(state, row_id, result); @@ -209,10 +210,10 @@ void StandardColumnData::CheckpointScan(ColumnSegment &segment, ColumnScanState validity.ScanCommittedRange(row_group_start, offset_in_row_group, count, scan_vector); } -void StandardColumnData::DeserializeColumn(Deserializer &deserializer) { - ColumnData::DeserializeColumn(deserializer); - deserializer.ReadObject(101, "validity", - [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer); }); +void StandardColumnData::DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) { + ColumnData::DeserializeColumn(deserializer, target_stats); + deserializer.ReadObject( + 101, "validity", [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer, target_stats); }); } void StandardColumnData::GetColumnSegmentInfo(duckdb::idx_t row_group_index, vector col_path, diff --git a/src/duckdb/src/storage/table/struct_column_data.cpp b/src/duckdb/src/storage/table/struct_column_data.cpp index 7061510df..5fca4ddae 100644 --- a/src/duckdb/src/storage/table/struct_column_data.cpp +++ b/src/duckdb/src/storage/table/struct_column_data.cpp @@ -5,6 +5,7 @@ #include "duckdb/storage/table/column_checkpoint_state.hpp" #include "duckdb/storage/table/append_state.hpp" #include "duckdb/storage/table/scan_state.hpp" +#include "duckdb/storage/table/update_segment.hpp" namespace duckdb { @@ -14,7 +15,7 @@ StructColumnData::StructColumnData(BlockManager &block_manager, DataTableInfo &i validity(block_manager, info, 0, start_row, *this) { D_ASSERT(type.InternalType() == PhysicalType::STRUCT); auto &child_types = StructType::GetChildTypes(type); - D_ASSERT(child_types.size() > 0); + D_ASSERT(!child_types.empty()); if (type.id() != LogicalTypeId::UNION && StructType::IsUnnamed(type)) { throw InvalidInputException("A table cannot be created from an unnamed struct"); } @@ -36,8 +37,25 @@ void StructColumnData::SetStart(idx_t new_start) { } bool StructColumnData::CheckZonemap(ColumnScanState &state, TableFilter &filter) { - // table filters are not supported yet for struct columns - return false; + if (!state.segment_checked) { + if (!state.current) { + return true; + } + state.segment_checked = true; + auto prune_result = filter.CheckStatistics(state.current->stats.statistics); + if (prune_result != FilterPropagateResult::FILTER_ALWAYS_FALSE) { + return true; + } + if (updates) { + auto update_stats = updates->GetStatistics(); + prune_result = filter.CheckStatistics(*update_stats); + return prune_result != FilterPropagateResult::FILTER_ALWAYS_FALSE; + } else { + return false; + } + } else { + return true; + } } idx_t StructColumnData::GetMaxEntry() { @@ -148,6 +166,7 @@ idx_t StructColumnData::Fetch(ColumnScanState &state, row_t row_id, Vector &resu // insert any child states that are required for (idx_t i = state.child_states.size(); i < child_entries.size() + 1; i++) { ColumnScanState child_state; + child_state.scan_options = state.scan_options; state.child_states.push_back(std::move(child_state)); } // fetch the validity state @@ -273,12 +292,13 @@ unique_ptr StructColumnData::Checkpoint(RowGroup &row_gro return std::move(checkpoint_state); } -void StructColumnData::DeserializeColumn(Deserializer &deserializer) { - deserializer.ReadObject(101, "validity", - [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer); }); +void StructColumnData::DeserializeColumn(Deserializer &deserializer, BaseStatistics &target_stats) { + deserializer.ReadObject( + 101, "validity", [&](Deserializer &deserializer) { validity.DeserializeColumn(deserializer, target_stats); }); deserializer.ReadList(102, "sub_columns", [&](Deserializer::List &list, idx_t i) { - list.ReadObject([&](Deserializer &item) { sub_columns[i]->DeserializeColumn(item); }); + auto &child_stats = StructStats::GetChildStats(target_stats, i); + list.ReadObject([&](Deserializer &item) { sub_columns[i]->DeserializeColumn(item, child_stats); }); }); this->count = validity.count; diff --git a/src/duckdb/src/storage/table/table_statistics.cpp b/src/duckdb/src/storage/table/table_statistics.cpp index 4df780f50..5d3a00e10 100644 --- a/src/duckdb/src/storage/table/table_statistics.cpp +++ b/src/duckdb/src/storage/table/table_statistics.cpp @@ -2,6 +2,7 @@ #include "duckdb/storage/table/persistent_table_data.hpp" #include "duckdb/common/serializer/serializer.hpp" #include "duckdb/common/serializer/deserializer.hpp" +#include "duckdb/execution/reservoir_sample.hpp" namespace duckdb { @@ -69,7 +70,10 @@ void TableStatistics::MergeStats(TableStatistics &other) { auto l = GetLock(); D_ASSERT(column_stats.size() == other.column_stats.size()); for (idx_t i = 0; i < column_stats.size(); i++) { - column_stats[i]->Merge(*other.column_stats[i]); + if (column_stats[i]) { + D_ASSERT(other.column_stats[i]); + column_stats[i]->Merge(*other.column_stats[i]); + } } } @@ -103,6 +107,7 @@ void TableStatistics::CopyStats(TableStatistics &other) { void TableStatistics::Serialize(Serializer &serializer) const { serializer.WriteProperty(100, "column_stats", column_stats); + serializer.WritePropertyWithDefault>(101, "table_sample", table_sample, nullptr); } void TableStatistics::Deserialize(Deserializer &deserializer, ColumnList &columns) { @@ -120,6 +125,7 @@ void TableStatistics::Deserialize(Deserializer &deserializer, ColumnList &column deserializer.Unset(); }); + table_sample = deserializer.ReadPropertyWithDefault>(101, "sample", nullptr); } unique_ptr TableStatistics::GetLock() { diff --git a/src/duckdb/src/storage/table/update_segment.cpp b/src/duckdb/src/storage/table/update_segment.cpp index c831e2a2b..1e1039961 100644 --- a/src/duckdb/src/storage/table/update_segment.cpp +++ b/src/duckdb/src/storage/table/update_segment.cpp @@ -6,7 +6,7 @@ #include "duckdb/transaction/duck_transaction.hpp" #include "duckdb/transaction/update_info.hpp" #include "duckdb/common/printer.hpp" - +#include "duckdb/common/exception/transaction_exception.hpp" #include namespace duckdb { @@ -143,6 +143,8 @@ static UpdateSegment::fetch_update_function_t GetFetchUpdateFunction(PhysicalTyp return UpdateMergeFetch; case PhysicalType::INT128: return UpdateMergeFetch; + case PhysicalType::UINT128: + return UpdateMergeFetch; case PhysicalType::FLOAT: return UpdateMergeFetch; case PhysicalType::DOUBLE: @@ -208,6 +210,8 @@ static UpdateSegment::fetch_committed_function_t GetFetchCommittedFunction(Physi return TemplatedFetchCommitted; case PhysicalType::INT128: return TemplatedFetchCommitted; + case PhysicalType::UINT128: + return TemplatedFetchCommitted; case PhysicalType::FLOAT: return TemplatedFetchCommitted; case PhysicalType::DOUBLE: @@ -304,6 +308,8 @@ static UpdateSegment::fetch_committed_range_function_t GetFetchCommittedRangeFun return TemplatedFetchCommittedRange; case PhysicalType::INT128: return TemplatedFetchCommittedRange; + case PhysicalType::UINT128: + return TemplatedFetchCommittedRange; case PhysicalType::FLOAT: return TemplatedFetchCommittedRange; case PhysicalType::DOUBLE: @@ -406,6 +412,8 @@ static UpdateSegment::fetch_row_function_t GetFetchRowFunction(PhysicalType type return TemplatedFetchRow; case PhysicalType::INT128: return TemplatedFetchRow; + case PhysicalType::UINT128: + return TemplatedFetchRow; case PhysicalType::FLOAT: return TemplatedFetchRow; case PhysicalType::DOUBLE: @@ -473,6 +481,8 @@ static UpdateSegment::rollback_update_function_t GetRollbackUpdateFunction(Physi return RollbackUpdate; case PhysicalType::INT128: return RollbackUpdate; + case PhysicalType::UINT128: + return RollbackUpdate; case PhysicalType::FLOAT: return RollbackUpdate; case PhysicalType::DOUBLE: @@ -660,6 +670,8 @@ static UpdateSegment::initialize_update_function_t GetInitializeUpdateFunction(P return InitializeUpdateData; case PhysicalType::INT128: return InitializeUpdateData; + case PhysicalType::UINT128: + return InitializeUpdateData; case PhysicalType::FLOAT: return InitializeUpdateData; case PhysicalType::DOUBLE: @@ -867,6 +879,8 @@ static UpdateSegment::merge_update_function_t GetMergeUpdateFunction(PhysicalTyp return MergeUpdateLoop; case PhysicalType::INT128: return MergeUpdateLoop; + case PhysicalType::UINT128: + return MergeUpdateLoop; case PhysicalType::FLOAT: return MergeUpdateLoop; case PhysicalType::DOUBLE: @@ -981,6 +995,8 @@ UpdateSegment::statistics_update_function_t GetStatisticsUpdateFunction(Physical return TemplatedUpdateNumericStatistics; case PhysicalType::INT128: return TemplatedUpdateNumericStatistics; + case PhysicalType::UINT128: + return TemplatedUpdateNumericStatistics; case PhysicalType::FLOAT: return TemplatedUpdateNumericStatistics; case PhysicalType::DOUBLE: diff --git a/src/duckdb/src/storage/table_index_list.cpp b/src/duckdb/src/storage/table_index_list.cpp index 2e562b753..0f5df1f0c 100644 --- a/src/duckdb/src/storage/table_index_list.cpp +++ b/src/duckdb/src/storage/table_index_list.cpp @@ -2,6 +2,11 @@ #include "duckdb/storage/data_table.hpp" #include "duckdb/common/types/conflict_manager.hpp" +#include "duckdb/execution/index/unknown_index.hpp" +#include "duckdb/execution/index/index_type_set.hpp" +#include "duckdb/storage/table/data_table_info.hpp" +#include "duckdb/main/database.hpp" +#include "duckdb/main/config.hpp" namespace duckdb { void TableIndexList::AddIndex(unique_ptr index) { @@ -50,6 +55,34 @@ bool TableIndexList::NameIsUnique(const string &name) { return true; } +void TableIndexList::InitializeIndexes(ClientContext &context, DataTableInfo &table_info) { + lock_guard lock(indexes_lock); + for (auto &index : indexes) { + if (!index->IsUnknown()) { + continue; + } + + auto &unknown_index = index->Cast(); + auto &index_type_name = unknown_index.GetIndexType(); + + // Do we know the type of this index now? + auto index_type = context.db->config.GetIndexTypes().FindByName(index_type_name); + if (!index_type) { + continue; + } + + // Swap this with a new index + auto &create_info = unknown_index.GetCreateInfo(); + auto &storage_info = unknown_index.GetStorageInfo(); + + auto index_instance = index_type->create_instance(create_info.index_name, create_info.constraint_type, + create_info.column_ids, unknown_index.unbound_expressions, + *table_info.table_io_manager, table_info.db, storage_info); + + index = std::move(index_instance); + } +} + bool TableIndexList::Empty() { lock_guard lock(indexes_lock); return indexes.empty(); diff --git a/src/duckdb/src/storage/temporary_memory_manager.cpp b/src/duckdb/src/storage/temporary_memory_manager.cpp new file mode 100644 index 000000000..227794505 --- /dev/null +++ b/src/duckdb/src/storage/temporary_memory_manager.cpp @@ -0,0 +1,146 @@ +#include "duckdb/storage/temporary_memory_manager.hpp" + +#include "duckdb/main/client_context.hpp" +#include "duckdb/parallel/task_scheduler.hpp" +#include "duckdb/storage/buffer_manager.hpp" + +namespace duckdb { + +TemporaryMemoryState::TemporaryMemoryState(TemporaryMemoryManager &temporary_memory_manager_p, + idx_t minimum_reservation_p) + : temporary_memory_manager(temporary_memory_manager_p), remaining_size(0), + minimum_reservation(minimum_reservation_p), reservation(0) { +} + +TemporaryMemoryState::~TemporaryMemoryState() { + temporary_memory_manager.Unregister(*this); +} + +void TemporaryMemoryState::SetRemainingSize(ClientContext &context, idx_t new_remaining_size) { + auto guard = temporary_memory_manager.Lock(); + temporary_memory_manager.SetRemainingSize(*this, new_remaining_size); + temporary_memory_manager.UpdateState(context, *this); +} + +idx_t TemporaryMemoryState::GetRemainingSize() const { + return remaining_size; +} + +void TemporaryMemoryState::SetMinimumReservation(idx_t new_minimum_reservation) { + minimum_reservation = new_minimum_reservation; +} + +idx_t TemporaryMemoryState::GetReservation() const { + return reservation; +} + +TemporaryMemoryManager::TemporaryMemoryManager() : reservation(0), remaining_size(0) { +} + +unique_lock TemporaryMemoryManager::Lock() { + return unique_lock(lock); +} + +void TemporaryMemoryManager::UpdateConfiguration(ClientContext &context) { + auto &buffer_manager = BufferManager::GetBufferManager(context); + auto &task_scheduler = TaskScheduler::GetScheduler(context); + + memory_limit = MAXIMUM_MEMORY_LIMIT_RATIO * double(buffer_manager.GetMaxMemory()); + has_temporary_directory = buffer_manager.HasTemporaryDirectory(); + num_threads = task_scheduler.NumberOfThreads(); +} + +TemporaryMemoryManager &TemporaryMemoryManager::Get(ClientContext &context) { + return BufferManager::GetBufferManager(context).GetTemporaryMemoryManager(); +} + +unique_ptr TemporaryMemoryManager::Register(ClientContext &context) { + auto guard = Lock(); + UpdateConfiguration(context); + + auto minimum_reservation = MinValue(num_threads * MINIMUM_RESERVATION_PER_STATE_PER_THREAD, + memory_limit / MINIMUM_RESERVATION_MEMORY_LIMIT_DIVISOR); + auto result = unique_ptr(new TemporaryMemoryState(*this, minimum_reservation)); + SetRemainingSize(*result, result->minimum_reservation); + SetReservation(*result, result->minimum_reservation); + active_states.insert(*result); + + Verify(); + return result; +} + +void TemporaryMemoryManager::UpdateState(ClientContext &context, TemporaryMemoryState &temporary_memory_state) { + UpdateConfiguration(context); + + if (context.config.force_external) { + // We're forcing external processing. Give it the minimum + SetReservation(temporary_memory_state, temporary_memory_state.minimum_reservation); + } else if (!has_temporary_directory) { + // We cannot offload, so we cannot limit memory usage. Set reservation equal to the remaining size + SetReservation(temporary_memory_state, temporary_memory_state.remaining_size); + } else if (reservation - temporary_memory_state.reservation >= memory_limit) { + // We overshot. Set reservation equal to the minimum + SetReservation(temporary_memory_state, temporary_memory_state.minimum_reservation); + } else { + // The lower bound for the reservation of this state is its minimum reservation + auto &lower_bound = temporary_memory_state.minimum_reservation; + + // The upper bound for the reservation of this state is the minimum of: + // 1. Remaining size of the state + // 2. MAXIMUM_FREE_MEMORY_RATIO * free memory + auto free_memory = memory_limit - (reservation - temporary_memory_state.reservation); + auto upper_bound = + MinValue(temporary_memory_state.remaining_size, MAXIMUM_FREE_MEMORY_RATIO * free_memory); + + if (remaining_size > memory_limit) { + // We're processing more data than fits in memory, so we must further limit memory usage. + // The upper bound for the reservation of this state is now also the minimum of: + // 3. The ratio of the remaining size of this state and the total remaining size * memory limit + auto ratio_of_remaining = double(temporary_memory_state.remaining_size) / double(remaining_size); + upper_bound = MinValue(upper_bound, ratio_of_remaining * memory_limit); + } + + SetReservation(temporary_memory_state, MaxValue(lower_bound, upper_bound)); + } + + Verify(); +} + +void TemporaryMemoryManager::SetRemainingSize(TemporaryMemoryState &temporary_memory_state, idx_t new_remaining_size) { + D_ASSERT(this->remaining_size >= temporary_memory_state.remaining_size); + this->remaining_size -= temporary_memory_state.remaining_size; + temporary_memory_state.remaining_size = new_remaining_size; + this->remaining_size += temporary_memory_state.remaining_size; +} + +void TemporaryMemoryManager::SetReservation(TemporaryMemoryState &temporary_memory_state, idx_t new_reservation) { + D_ASSERT(this->reservation >= temporary_memory_state.reservation); + this->reservation -= temporary_memory_state.reservation; + temporary_memory_state.reservation = new_reservation; + this->reservation += temporary_memory_state.reservation; +} + +void TemporaryMemoryManager::Unregister(TemporaryMemoryState &temporary_memory_state) { + auto guard = Lock(); + + SetReservation(temporary_memory_state, 0); + SetRemainingSize(temporary_memory_state, 0); + active_states.erase(temporary_memory_state); + + Verify(); +} + +void TemporaryMemoryManager::Verify() const { +#ifdef DEBUG + idx_t total_reservation = 0; + idx_t total_remaining_size = 0; + for (auto &temporary_memory_state : active_states) { + total_reservation += temporary_memory_state.get().reservation; + total_remaining_size += temporary_memory_state.get().remaining_size; + } + D_ASSERT(total_reservation == this->reservation); + D_ASSERT(total_remaining_size == this->remaining_size); +#endif +} + +} // namespace duckdb diff --git a/src/duckdb/src/storage/wal_replay.cpp b/src/duckdb/src/storage/wal_replay.cpp index 7908b3d41..0ffd1502c 100644 --- a/src/duckdb/src/storage/wal_replay.cpp +++ b/src/duckdb/src/storage/wal_replay.cpp @@ -8,7 +8,6 @@ #include "duckdb/common/serializer/binary_deserializer.hpp" #include "duckdb/common/serializer/buffered_file_reader.hpp" #include "duckdb/common/string_util.hpp" -#include "duckdb/execution/index/art/art.hpp" #include "duckdb/main/attached_database.hpp" #include "duckdb/main/client_context.hpp" #include "duckdb/main/connection.hpp" @@ -22,9 +21,143 @@ #include "duckdb/planner/parsed_data/bound_create_table_info.hpp" #include "duckdb/storage/storage_manager.hpp" #include "duckdb/storage/write_ahead_log.hpp" +#include "duckdb/common/serializer/memory_stream.hpp" +#include "duckdb/common/checksum.hpp" +#include "duckdb/execution/index/index_type_set.hpp" +#include "duckdb/execution/index/art/art.hpp" namespace duckdb { +class ReplayState { +public: + ReplayState(AttachedDatabase &db, ClientContext &context) : db(db), context(context), catalog(db.GetCatalog()) { + } + + AttachedDatabase &db; + ClientContext &context; + Catalog &catalog; + optional_ptr current_table; + MetaBlockPointer checkpoint_id; + idx_t wal_version = 1; +}; + +class WriteAheadLogDeserializer { +public: + WriteAheadLogDeserializer(ReplayState &state_p, BufferedFileReader &stream_p, bool deserialize_only = false) + : state(state_p), db(state.db), context(state.context), catalog(state.catalog), data(nullptr), + stream(nullptr, 0), deserializer(stream_p), deserialize_only(deserialize_only) { + } + WriteAheadLogDeserializer(ReplayState &state_p, unique_ptr data_p, idx_t size, + bool deserialize_only = false) + : state(state_p), db(state.db), context(state.context), catalog(state.catalog), data(std::move(data_p)), + stream(data.get(), size), deserializer(stream), deserialize_only(deserialize_only) { + } + + static WriteAheadLogDeserializer Open(ReplayState &state_p, BufferedFileReader &stream, + bool deserialize_only = false) { + if (state_p.wal_version == 1) { + // old WAL versions do not have checksums + return WriteAheadLogDeserializer(state_p, stream, deserialize_only); + } + if (state_p.wal_version != 2) { + throw IOException("Failed to read WAL of version %llu - can only read version 1 and 2", + state_p.wal_version); + } + // read the checksum and size + auto size = stream.Read(); + auto stored_checksum = stream.Read(); + auto offset = stream.CurrentOffset(); + auto file_size = stream.FileSize(); + + if (offset + size > file_size) { + throw SerializationException( + "Corrupt WAL file: entry size exceeded remaining data in file at byte position %llu " + "(found entry with size %llu bytes, file size %llu bytes)", + offset, size, file_size); + } + + // allocate a buffer and read data into the buffer + auto buffer = unique_ptr(new data_t[size]); + stream.ReadData(buffer.get(), size); + + // compute and verify the checksum + auto computed_checksum = Checksum(buffer.get(), size); + if (stored_checksum != computed_checksum) { + throw SerializationException( + "Corrupt WAL file: entry at byte position %llu computed checksum %llu does not match " + "stored checksum %llu", + offset, computed_checksum, stored_checksum); + } + return WriteAheadLogDeserializer(state_p, std::move(buffer), size, deserialize_only); + } + + bool ReplayEntry() { + deserializer.Begin(); + auto wal_type = deserializer.ReadProperty(100, "wal_type"); + if (wal_type == WALType::WAL_FLUSH) { + deserializer.End(); + return true; + } + ReplayEntry(wal_type); + deserializer.End(); + return false; + } + + bool DeserializeOnly() { + return deserialize_only; + } + +protected: + void ReplayEntry(WALType wal_type); + + void ReplayVersion(); + + void ReplayCreateTable(); + void ReplayDropTable(); + void ReplayAlter(); + + void ReplayCreateView(); + void ReplayDropView(); + + void ReplayCreateSchema(); + void ReplayDropSchema(); + + void ReplayCreateType(); + void ReplayDropType(); + + void ReplayCreateSequence(); + void ReplayDropSequence(); + void ReplaySequenceValue(); + + void ReplayCreateMacro(); + void ReplayDropMacro(); + + void ReplayCreateTableMacro(); + void ReplayDropTableMacro(); + + void ReplayCreateIndex(); + void ReplayDropIndex(); + + void ReplayUseTable(); + void ReplayInsert(); + void ReplayDelete(); + void ReplayUpdate(); + void ReplayCheckpoint(); + +private: + ReplayState &state; + AttachedDatabase &db; + ClientContext &context; + Catalog &catalog; + unique_ptr data; + MemoryStream stream; + BinaryDeserializer deserializer; + bool deserialize_only; +}; + +//===--------------------------------------------------------------------===// +// Replay +//===--------------------------------------------------------------------===// bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { Connection con(database.GetDatabase()); auto initial_source = make_uniq(FileSystem::Get(database), path.c_str()); @@ -38,32 +171,27 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { // first deserialize the WAL to look for a checkpoint flag // if there is a checkpoint flag, we might have already flushed the contents of the WAL to disk ReplayState checkpoint_state(database, *con.context); - checkpoint_state.deserialize_only = true; try { while (true) { - // read the current entry - BinaryDeserializer deserializer(*initial_source); - deserializer.Begin(); - auto entry_type = deserializer.ReadProperty(100, "wal_type"); - if (entry_type == WALType::WAL_FLUSH) { - deserializer.End(); + // read the current entry (deserialize only) + auto deserializer = WriteAheadLogDeserializer::Open(checkpoint_state, *initial_source, true); + if (deserializer.ReplayEntry()) { // check if the file is exhausted if (initial_source->Finished()) { // we finished reading the file: break break; } - } else { - // replay the entry - checkpoint_state.ReplayEntry(entry_type, deserializer); - deserializer.End(); } } - } catch (SerializationException &ex) { // LCOV_EXCL_START - // serialization exception - torn WAL - // continue reading - } catch (std::exception &ex) { - Printer::PrintF("Exception in WAL playback during initial read: %s\n", ex.what()); - return false; + } catch (std::exception &ex) { // LCOV_EXCL_START + ErrorData error(ex); + if (error.Type() == ExceptionType::SERIALIZATION) { + // serialization exception - torn WAL + // continue reading + } else { + Printer::PrintF("Exception in WAL playback during initial read: %s\n", error.RawMessage()); + return false; + } } catch (...) { Printer::Print("Unknown Exception in WAL playback during initial read"); return false; @@ -90,11 +218,8 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { try { while (true) { // read the current entry - BinaryDeserializer deserializer(reader); - deserializer.Begin(); - auto entry_type = deserializer.ReadProperty(100, "wal_type"); - if (entry_type == WALType::WAL_FLUSH) { - deserializer.End(); + auto deserializer = WriteAheadLogDeserializer::Open(state, reader); + if (deserializer.ReplayEntry()) { con.Commit(); // check if the file is exhausted if (reader.Finished()) { @@ -102,19 +227,15 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { break; } con.BeginTransaction(); - } else { - // replay the entry - state.ReplayEntry(entry_type, deserializer); - deserializer.End(); } } - } catch (SerializationException &ex) { // LCOV_EXCL_START - // serialization error during WAL replay: rollback - con.Rollback(); - } catch (std::exception &ex) { - // FIXME: this should report a proper warning in the connection - Printer::PrintF("Exception in WAL playback: %s\n", ex.what()); - // exception thrown in WAL replay: rollback + } catch (std::exception &ex) { // LCOV_EXCL_START + ErrorData error(ex); + if (error.Type() != ExceptionType::SERIALIZATION) { + // FIXME: this should report a proper warning in the connection + Printer::PrintF("Exception in WAL playback: %s\n", error.RawMessage()); + // exception thrown in WAL replay: rollback + } con.Rollback(); } catch (...) { Printer::Print("Unknown Exception in WAL playback: %s\n"); @@ -127,88 +248,98 @@ bool WriteAheadLog::Replay(AttachedDatabase &database, string &path) { //===--------------------------------------------------------------------===// // Replay Entries //===--------------------------------------------------------------------===// -void ReplayState::ReplayEntry(WALType entry_type, BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayEntry(WALType entry_type) { switch (entry_type) { + case WALType::WAL_VERSION: + ReplayVersion(); + break; case WALType::CREATE_TABLE: - ReplayCreateTable(deserializer); + ReplayCreateTable(); break; case WALType::DROP_TABLE: - ReplayDropTable(deserializer); + ReplayDropTable(); break; case WALType::ALTER_INFO: - ReplayAlter(deserializer); + ReplayAlter(); break; case WALType::CREATE_VIEW: - ReplayCreateView(deserializer); + ReplayCreateView(); break; case WALType::DROP_VIEW: - ReplayDropView(deserializer); + ReplayDropView(); break; case WALType::CREATE_SCHEMA: - ReplayCreateSchema(deserializer); + ReplayCreateSchema(); break; case WALType::DROP_SCHEMA: - ReplayDropSchema(deserializer); + ReplayDropSchema(); break; case WALType::CREATE_SEQUENCE: - ReplayCreateSequence(deserializer); + ReplayCreateSequence(); break; case WALType::DROP_SEQUENCE: - ReplayDropSequence(deserializer); + ReplayDropSequence(); break; case WALType::SEQUENCE_VALUE: - ReplaySequenceValue(deserializer); + ReplaySequenceValue(); break; case WALType::CREATE_MACRO: - ReplayCreateMacro(deserializer); + ReplayCreateMacro(); break; case WALType::DROP_MACRO: - ReplayDropMacro(deserializer); + ReplayDropMacro(); break; case WALType::CREATE_TABLE_MACRO: - ReplayCreateTableMacro(deserializer); + ReplayCreateTableMacro(); break; case WALType::DROP_TABLE_MACRO: - ReplayDropTableMacro(deserializer); + ReplayDropTableMacro(); break; case WALType::CREATE_INDEX: - ReplayCreateIndex(deserializer); + ReplayCreateIndex(); break; case WALType::DROP_INDEX: - ReplayDropIndex(deserializer); + ReplayDropIndex(); break; case WALType::USE_TABLE: - ReplayUseTable(deserializer); + ReplayUseTable(); break; case WALType::INSERT_TUPLE: - ReplayInsert(deserializer); + ReplayInsert(); break; case WALType::DELETE_TUPLE: - ReplayDelete(deserializer); + ReplayDelete(); break; case WALType::UPDATE_TUPLE: - ReplayUpdate(deserializer); + ReplayUpdate(); break; case WALType::CHECKPOINT: - ReplayCheckpoint(deserializer); + ReplayCheckpoint(); break; case WALType::CREATE_TYPE: - ReplayCreateType(deserializer); + ReplayCreateType(); break; case WALType::DROP_TYPE: - ReplayDropType(deserializer); + ReplayDropType(); break; default: throw InternalException("Invalid WAL entry type!"); } } +//===--------------------------------------------------------------------===// +// Replay Version +//===--------------------------------------------------------------------===// +void WriteAheadLogDeserializer::ReplayVersion() { + state.wal_version = deserializer.ReadProperty(101, "version"); +} + //===--------------------------------------------------------------------===// // Replay Table //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateTable(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateTable() { auto info = deserializer.ReadProperty>(101, "table"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } // bind the constraints to the table again @@ -219,25 +350,23 @@ void ReplayState::ReplayCreateTable(BinaryDeserializer &deserializer) { catalog.CreateTable(context, *bound_info); } -void ReplayState::ReplayDropTable(BinaryDeserializer &deserializer) { - +void WriteAheadLogDeserializer::ReplayDropTable() { DropInfo info; info.type = CatalogType::TABLE_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.DropEntry(context, info); } -void ReplayState::ReplayAlter(BinaryDeserializer &deserializer) { - +void WriteAheadLogDeserializer::ReplayAlter() { auto info = deserializer.ReadProperty>(101, "info"); auto &alter_info = info->Cast(); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.Alter(context, alter_info); @@ -246,20 +375,20 @@ void ReplayState::ReplayAlter(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay View //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateView(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateView() { auto entry = deserializer.ReadProperty>(101, "view"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.CreateView(context, entry->Cast()); } -void ReplayState::ReplayDropView(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropView() { DropInfo info; info.type = CatalogType::VIEW_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.DropEntry(context, info); @@ -268,22 +397,22 @@ void ReplayState::ReplayDropView(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Schema //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateSchema(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateSchema() { CreateSchemaInfo info; info.schema = deserializer.ReadProperty(101, "schema"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.CreateSchema(context, info); } -void ReplayState::ReplayDropSchema(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropSchema() { DropInfo info; info.type = CatalogType::SCHEMA_ENTRY; info.name = deserializer.ReadProperty(101, "schema"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } @@ -293,19 +422,19 @@ void ReplayState::ReplayDropSchema(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Custom Type //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateType(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateType() { auto info = deserializer.ReadProperty>(101, "type"); info->on_conflict = OnCreateConflict::IGNORE_ON_CONFLICT; catalog.CreateType(context, info->Cast()); } -void ReplayState::ReplayDropType(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropType() { DropInfo info; info.type = CatalogType::TYPE_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } @@ -315,62 +444,59 @@ void ReplayState::ReplayDropType(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Sequence //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateSequence(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateSequence() { auto entry = deserializer.ReadProperty>(101, "sequence"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.CreateSequence(context, entry->Cast()); } -void ReplayState::ReplayDropSequence(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropSequence() { DropInfo info; info.type = CatalogType::SEQUENCE_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.DropEntry(context, info); } -void ReplayState::ReplaySequenceValue(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplaySequenceValue() { auto schema = deserializer.ReadProperty(101, "schema"); auto name = deserializer.ReadProperty(102, "name"); auto usage_count = deserializer.ReadProperty(103, "usage_count"); auto counter = deserializer.ReadProperty(104, "counter"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } // fetch the sequence from the catalog auto &seq = catalog.GetEntry(context, schema, name); - if (usage_count > seq.usage_count) { - seq.usage_count = usage_count; - seq.counter = counter; - } + seq.ReplayValue(usage_count, counter); } //===--------------------------------------------------------------------===// // Replay Macro //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateMacro(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateMacro() { auto entry = deserializer.ReadProperty>(101, "macro"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.CreateFunction(context, entry->Cast()); } -void ReplayState::ReplayDropMacro(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropMacro() { DropInfo info; info.type = CatalogType::MACRO_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } @@ -380,20 +506,20 @@ void ReplayState::ReplayDropMacro(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Table Macro //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateTableMacro(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayCreateTableMacro() { auto entry = deserializer.ReadProperty>(101, "table_macro"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } catalog.CreateFunction(context, entry->Cast()); } -void ReplayState::ReplayDropTableMacro(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropTableMacro() { DropInfo info; info.type = CatalogType::TABLE_MACRO_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } @@ -403,8 +529,7 @@ void ReplayState::ReplayDropTableMacro(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Index //===--------------------------------------------------------------------===// -void ReplayState::ReplayCreateIndex(BinaryDeserializer &deserializer) { - +void WriteAheadLogDeserializer::ReplayCreateIndex() { auto create_info = deserializer.ReadProperty>(101, "index_catalog_entry"); auto index_info = deserializer.ReadProperty(102, "index_storage_info"); D_ASSERT(index_info.IsValid() && !index_info.name.empty()); @@ -436,15 +561,25 @@ void ReplayState::ReplayCreateIndex(BinaryDeserializer &deserializer) { } }); - if (deserialize_only) { + if (DeserializeOnly()) { return; } auto &info = create_info->Cast(); + // Ensure the index type exists + if (info.index_type.empty()) { + info.index_type = ART::TYPE_NAME; + } + + auto index_type = context.db->config.GetIndexTypes().FindByName(info.index_type); + if (!index_type) { + throw InternalException("Index type \"%s\" not recognized", info.index_type); + } + // create the index in the catalog auto &table = catalog.GetEntry(context, create_info->schema, info.table).Cast(); auto &index = catalog.CreateIndex(context, info)->Cast(); - index.info = table.GetStorage().info; + index.info = make_shared(table.GetStorage().info, index.name); // insert the parsed expressions into the index so that we can (de)serialize them during consecutive checkpoints for (auto &parsed_expr : info.parsed_expressions) { @@ -480,17 +615,18 @@ void ReplayState::ReplayCreateIndex(BinaryDeserializer &deserializer) { } auto &data_table = table.GetStorage(); - auto art = make_uniq(info.index_name, info.constraint_type, info.column_ids, TableIOManager::Get(data_table), - std::move(unbound_expressions), data_table.db, nullptr, index_info); - data_table.info->indexes.AddIndex(std::move(art)); + auto index_instance = + index_type->create_instance(info.index_name, info.constraint_type, info.column_ids, unbound_expressions, + TableIOManager::Get(data_table), data_table.db, index_info); + data_table.info->indexes.AddIndex(std::move(index_instance)); } -void ReplayState::ReplayDropIndex(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDropIndex() { DropInfo info; info.type = CatalogType::INDEX_ENTRY; info.schema = deserializer.ReadProperty(101, "schema"); info.name = deserializer.ReadProperty(102, "name"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } @@ -500,36 +636,36 @@ void ReplayState::ReplayDropIndex(BinaryDeserializer &deserializer) { //===--------------------------------------------------------------------===// // Replay Data //===--------------------------------------------------------------------===// -void ReplayState::ReplayUseTable(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayUseTable() { auto schema_name = deserializer.ReadProperty(101, "schema"); auto table_name = deserializer.ReadProperty(102, "table"); - if (deserialize_only) { + if (DeserializeOnly()) { return; } - current_table = &catalog.GetEntry(context, schema_name, table_name); + state.current_table = &catalog.GetEntry(context, schema_name, table_name); } -void ReplayState::ReplayInsert(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayInsert() { DataChunk chunk; deserializer.ReadObject(101, "chunk", [&](Deserializer &object) { chunk.Deserialize(object); }); - if (deserialize_only) { + if (DeserializeOnly()) { return; } - if (!current_table) { - throw Exception("Corrupt WAL: insert without table"); + if (!state.current_table) { + throw InternalException("Corrupt WAL: insert without table"); } // append to the current table - current_table->GetStorage().LocalAppend(*current_table, context, chunk); + state.current_table->GetStorage().LocalAppend(*state.current_table, context, chunk); } -void ReplayState::ReplayDelete(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayDelete() { DataChunk chunk; deserializer.ReadObject(101, "chunk", [&](Deserializer &object) { chunk.Deserialize(object); }); - if (deserialize_only) { + if (DeserializeOnly()) { return; } - if (!current_table) { + if (!state.current_table) { throw InternalException("Corrupt WAL: delete without table"); } @@ -541,24 +677,24 @@ void ReplayState::ReplayDelete(BinaryDeserializer &deserializer) { // delete the tuples from the current table for (idx_t i = 0; i < chunk.size(); i++) { row_ids[0] = source_ids[i]; - current_table->GetStorage().Delete(*current_table, context, row_identifiers, 1); + state.current_table->GetStorage().Delete(*state.current_table, context, row_identifiers, 1); } } -void ReplayState::ReplayUpdate(BinaryDeserializer &deserializer) { +void WriteAheadLogDeserializer::ReplayUpdate() { auto column_path = deserializer.ReadProperty>(101, "column_indexes"); DataChunk chunk; deserializer.ReadObject(102, "chunk", [&](Deserializer &object) { chunk.Deserialize(object); }); - if (deserialize_only) { + if (DeserializeOnly()) { return; } - if (!current_table) { + if (!state.current_table) { throw InternalException("Corrupt WAL: update without table"); } - if (column_path[0] >= current_table->GetColumns().PhysicalColumnCount()) { + if (column_path[0] >= state.current_table->GetColumns().PhysicalColumnCount()) { throw InternalException("Corrupt WAL: column index for update out of bounds"); } @@ -567,11 +703,11 @@ void ReplayState::ReplayUpdate(BinaryDeserializer &deserializer) { chunk.data.pop_back(); // now perform the update - current_table->GetStorage().UpdateColumn(*current_table, context, row_ids, column_path, chunk); + state.current_table->GetStorage().UpdateColumn(*state.current_table, context, row_ids, column_path, chunk); } -void ReplayState::ReplayCheckpoint(BinaryDeserializer &deserializer) { - checkpoint_id = deserializer.ReadProperty(101, "meta_block"); +void WriteAheadLogDeserializer::ReplayCheckpoint() { + state.checkpoint_id = deserializer.ReadProperty(101, "meta_block"); } } // namespace duckdb diff --git a/src/duckdb/src/storage/write_ahead_log.cpp b/src/duckdb/src/storage/write_ahead_log.cpp index e3cb72235..a90b9dedf 100644 --- a/src/duckdb/src/storage/write_ahead_log.cpp +++ b/src/duckdb/src/storage/write_ahead_log.cpp @@ -12,9 +12,13 @@ #include "duckdb/storage/index.hpp" #include "duckdb/storage/table/data_table_info.hpp" #include "duckdb/storage/table_io_manager.hpp" +#include "duckdb/common/checksum.hpp" +#include "duckdb/common/serializer/memory_stream.hpp" namespace duckdb { +const uint64_t WAL_VERSION_NUMBER = 2; + WriteAheadLog::WriteAheadLog(AttachedDatabase &database, const string &path) : skip_writing(false), database(database) { wal_path = path; writer = make_uniq(FileSystem::Get(database), path.c_str(), @@ -49,13 +53,107 @@ void WriteAheadLog::Delete() { fs.RemoveFile(wal_path); } +//===--------------------------------------------------------------------===// +// Serializer +//===--------------------------------------------------------------------===// +class ChecksumWriter : public WriteStream { +public: + explicit ChecksumWriter(WriteAheadLog &wal) : wal(wal), stream(wal.GetWriter()) { + } + + void WriteData(const_data_ptr_t buffer, idx_t write_size) override { + if (wal.skip_writing) { + return; + } + // buffer data into the memory stream + memory_stream.WriteData(buffer, write_size); + } + + void Flush() { + if (wal.skip_writing) { + return; + } + auto data = memory_stream.GetData(); + auto size = memory_stream.GetPosition(); + // compute the checksum over the entry + auto checksum = Checksum(data, size); + // write the checksum and the length of the entry + stream.Write(size); + stream.Write(checksum); + // write data to the underlying stream + stream.WriteData(memory_stream.GetData(), memory_stream.GetPosition()); + // rewind the buffer + memory_stream.Rewind(); + } + +private: + WriteAheadLog &wal; + WriteStream &stream; + MemoryStream memory_stream; +}; + +class WriteAheadLogSerializer { +public: + WriteAheadLogSerializer(WriteAheadLog &wal, WALType wal_type) + : wal(wal), checksum_writer(wal), serializer(checksum_writer) { + if (wal.skip_writing) { + return; + } + // write a version marker if none has been written yet + wal.WriteVersion(); + serializer.Begin(); + serializer.WriteProperty(100, "wal_type", wal_type); + } + + void End() { + if (wal.skip_writing) { + return; + } + serializer.End(); + checksum_writer.Flush(); + } + + template + void WriteProperty(const field_id_t field_id, const char *tag, const T &value) { + if (wal.skip_writing) { + return; + } + serializer.WriteProperty(field_id, tag, value); + } + + template + void WriteList(const field_id_t field_id, const char *tag, idx_t count, FUNC func) { + if (wal.skip_writing) { + return; + } + serializer.WriteList(field_id, tag, count, func); + } + +private: + WriteAheadLog &wal; + ChecksumWriter checksum_writer; + BinarySerializer serializer; +}; + //===--------------------------------------------------------------------===// // Write Entries //===--------------------------------------------------------------------===// -void WriteAheadLog::WriteCheckpoint(MetaBlockPointer meta_block) { +void WriteAheadLog::WriteVersion() { + if (writer->GetFileSize() > 0) { + // already written - no need to write a version marker + return; + } + // write the version marker + // note that we explicitly do not checksum the version entry BinarySerializer serializer(*writer); serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CHECKPOINT); + serializer.WriteProperty(100, "wal_type", WALType::WAL_VERSION); + serializer.WriteProperty(101, "version", idx_t(WAL_VERSION_NUMBER)); + serializer.End(); +} + +void WriteAheadLog::WriteCheckpoint(MetaBlockPointer meta_block) { + WriteAheadLogSerializer serializer(*this, WALType::CHECKPOINT); serializer.WriteProperty(101, "meta_block", meta_block); serializer.End(); } @@ -64,12 +162,7 @@ void WriteAheadLog::WriteCheckpoint(MetaBlockPointer meta_block) { // CREATE TABLE //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateTable(const TableCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_TABLE); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_TABLE); serializer.WriteProperty(101, "table", &entry); serializer.End(); } @@ -78,12 +171,7 @@ void WriteAheadLog::WriteCreateTable(const TableCatalogEntry &entry) { // DROP TABLE //===--------------------------------------------------------------------===// void WriteAheadLog::WriteDropTable(const TableCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_TABLE); + WriteAheadLogSerializer serializer(*this, WALType::DROP_TABLE); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); @@ -93,12 +181,7 @@ void WriteAheadLog::WriteDropTable(const TableCatalogEntry &entry) { // CREATE SCHEMA //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateSchema(const SchemaCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_SCHEMA); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_SCHEMA); serializer.WriteProperty(101, "schema", entry.name); serializer.End(); } @@ -107,35 +190,20 @@ void WriteAheadLog::WriteCreateSchema(const SchemaCatalogEntry &entry) { // SEQUENCES //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateSequence(const SequenceCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_SEQUENCE); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_SEQUENCE); serializer.WriteProperty(101, "sequence", &entry); serializer.End(); } void WriteAheadLog::WriteDropSequence(const SequenceCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_SEQUENCE); + WriteAheadLogSerializer serializer(*this, WALType::DROP_SEQUENCE); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); } void WriteAheadLog::WriteSequenceValue(const SequenceCatalogEntry &entry, SequenceValue val) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::SEQUENCE_VALUE); + WriteAheadLogSerializer serializer(*this, WALType::SEQUENCE_VALUE); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.WriteProperty(103, "usage_count", val.usage_count); @@ -147,46 +215,26 @@ void WriteAheadLog::WriteSequenceValue(const SequenceCatalogEntry &entry, Sequen // MACROS //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateMacro(const ScalarMacroCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_MACRO); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_MACRO); serializer.WriteProperty(101, "macro", &entry); serializer.End(); } void WriteAheadLog::WriteDropMacro(const ScalarMacroCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_MACRO); + WriteAheadLogSerializer serializer(*this, WALType::DROP_MACRO); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); } void WriteAheadLog::WriteCreateTableMacro(const TableMacroCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_TABLE_MACRO); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_TABLE_MACRO); serializer.WriteProperty(101, "table", &entry); serializer.End(); } void WriteAheadLog::WriteDropTableMacro(const TableMacroCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_TABLE_MACRO); + WriteAheadLogSerializer serializer(*this, WALType::DROP_TABLE_MACRO); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); @@ -196,7 +244,7 @@ void WriteAheadLog::WriteDropTableMacro(const TableMacroCatalogEntry &entry) { // Indexes //===--------------------------------------------------------------------===// -void SerializeIndexToWAL(BinarySerializer &serializer, const unique_ptr &index) { +void SerializeIndexToWAL(WriteAheadLogSerializer &serializer, const unique_ptr &index) { auto index_storage_info = index->GetStorageInfo(true); serializer.WriteProperty(102, "index_storage_info", index_storage_info); @@ -214,14 +262,12 @@ void WriteAheadLog::WriteCreateIndex(const IndexCatalogEntry &entry) { return; } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_INDEX); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_INDEX); serializer.WriteProperty(101, "index_catalog_entry", &entry); // now serialize the index data to the persistent storage and write the index metadata auto &duck_index_entry = entry.Cast(); - auto &indexes = duck_index_entry.info->indexes.Indexes(); + auto &indexes = duck_index_entry.GetDataTableInfo().indexes.Indexes(); // get the matching index and serialize its storage info for (auto const &index : indexes) { @@ -235,12 +281,7 @@ void WriteAheadLog::WriteCreateIndex(const IndexCatalogEntry &entry) { } void WriteAheadLog::WriteDropIndex(const IndexCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_INDEX); + WriteAheadLogSerializer serializer(*this, WALType::DROP_INDEX); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); @@ -250,23 +291,13 @@ void WriteAheadLog::WriteDropIndex(const IndexCatalogEntry &entry) { // Custom Types //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateType(const TypeCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_TYPE); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_TYPE); serializer.WriteProperty(101, "type", &entry); serializer.End(); } void WriteAheadLog::WriteDropType(const TypeCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_TYPE); + WriteAheadLogSerializer serializer(*this, WALType::DROP_TYPE); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); @@ -276,23 +307,13 @@ void WriteAheadLog::WriteDropType(const TypeCatalogEntry &entry) { // VIEWS //===--------------------------------------------------------------------===// void WriteAheadLog::WriteCreateView(const ViewCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::CREATE_VIEW); + WriteAheadLogSerializer serializer(*this, WALType::CREATE_VIEW); serializer.WriteProperty(101, "view", &entry); serializer.End(); } void WriteAheadLog::WriteDropView(const ViewCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_VIEW); + WriteAheadLogSerializer serializer(*this, WALType::DROP_VIEW); serializer.WriteProperty(101, "schema", entry.schema.name); serializer.WriteProperty(102, "name", entry.name); serializer.End(); @@ -302,12 +323,7 @@ void WriteAheadLog::WriteDropView(const ViewCatalogEntry &entry) { // DROP SCHEMA //===--------------------------------------------------------------------===// void WriteAheadLog::WriteDropSchema(const SchemaCatalogEntry &entry) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DROP_SCHEMA); + WriteAheadLogSerializer serializer(*this, WALType::DROP_SCHEMA); serializer.WriteProperty(101, "schema", entry.name); serializer.End(); } @@ -316,58 +332,38 @@ void WriteAheadLog::WriteDropSchema(const SchemaCatalogEntry &entry) { // DATA //===--------------------------------------------------------------------===// void WriteAheadLog::WriteSetTable(string &schema, string &table) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::USE_TABLE); + WriteAheadLogSerializer serializer(*this, WALType::USE_TABLE); serializer.WriteProperty(101, "schema", schema); serializer.WriteProperty(102, "table", table); serializer.End(); } void WriteAheadLog::WriteInsert(DataChunk &chunk) { - if (skip_writing) { - return; - } D_ASSERT(chunk.size() > 0); chunk.Verify(); - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::INSERT_TUPLE); + WriteAheadLogSerializer serializer(*this, WALType::INSERT_TUPLE); serializer.WriteProperty(101, "chunk", chunk); serializer.End(); } void WriteAheadLog::WriteDelete(DataChunk &chunk) { - if (skip_writing) { - return; - } D_ASSERT(chunk.size() > 0); D_ASSERT(chunk.ColumnCount() == 1 && chunk.data[0].GetType() == LogicalType::ROW_TYPE); chunk.Verify(); - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::DELETE_TUPLE); + WriteAheadLogSerializer serializer(*this, WALType::DELETE_TUPLE); serializer.WriteProperty(101, "chunk", chunk); serializer.End(); } void WriteAheadLog::WriteUpdate(DataChunk &chunk, const vector &column_indexes) { - if (skip_writing) { - return; - } D_ASSERT(chunk.size() > 0); D_ASSERT(chunk.ColumnCount() == 2); D_ASSERT(chunk.data[1].GetType().id() == LogicalType::ROW_TYPE); chunk.Verify(); - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::UPDATE_TUPLE); + WriteAheadLogSerializer serializer(*this, WALType::UPDATE_TUPLE); serializer.WriteProperty(101, "column_indexes", column_indexes); serializer.WriteProperty(102, "chunk", chunk); serializer.End(); @@ -377,12 +373,7 @@ void WriteAheadLog::WriteUpdate(DataChunk &chunk, const vector &column // Write ALTER Statement //===--------------------------------------------------------------------===// void WriteAheadLog::WriteAlter(const AlterInfo &info) { - if (skip_writing) { - return; - } - BinarySerializer serializer(*writer); - serializer.Begin(); - serializer.WriteProperty(100, "wal_type", WALType::ALTER_INFO); + WriteAheadLogSerializer serializer(*this, WALType::ALTER_INFO); serializer.WriteProperty(101, "info", &info); serializer.End(); } @@ -395,10 +386,8 @@ void WriteAheadLog::Flush() { return; } - BinarySerializer serializer(*writer); - serializer.Begin(); // write an empty entry - serializer.WriteProperty(100, "wal_type", WALType::WAL_FLUSH); + WriteAheadLogSerializer serializer(*this, WALType::WAL_FLUSH); serializer.End(); // flushes all changes made to the WAL to disk diff --git a/src/duckdb/src/transaction/commit_state.cpp b/src/duckdb/src/transaction/commit_state.cpp index 5a09d460d..58d00267f 100644 --- a/src/duckdb/src/transaction/commit_state.cpp +++ b/src/duckdb/src/transaction/commit_state.cpp @@ -171,6 +171,9 @@ void CommitState::WriteCatalogEntry(CatalogEntry &entry, data_ptr_t dataptr) { case CatalogType::PRAGMA_FUNCTION_ENTRY: case CatalogType::COLLATION_ENTRY: case CatalogType::DEPENDENCY_ENTRY: + case CatalogType::SECRET_ENTRY: + case CatalogType::SECRET_TYPE_ENTRY: + case CatalogType::SECRET_FUNCTION_ENTRY: // do nothing, these entries are not persisted to disk break; default: @@ -269,6 +272,7 @@ void CommitState::CommitEntry(UndoFlags type, data_ptr_t data) { if (!StringUtil::CIEquals(catalog_entry->name, catalog_entry->Parent().name)) { catalog_entry->set->UpdateTimestamp(*catalog_entry, commit_id); } + if (HAS_LOG) { // push the catalog update to the WAL WriteCatalogEntry(*catalog_entry, data + sizeof(CatalogEntry *)); diff --git a/src/duckdb/src/transaction/duck_transaction.cpp b/src/duckdb/src/transaction/duck_transaction.cpp index 89604a69c..528d718b7 100644 --- a/src/duckdb/src/transaction/duck_transaction.cpp +++ b/src/duckdb/src/transaction/duck_transaction.cpp @@ -111,7 +111,7 @@ bool DuckTransaction::AutomaticCheckpoint(AttachedDatabase &db) { return storage_manager.AutomaticCheckpoint(storage->EstimatedSize() + undo_buffer.EstimatedSize()); } -string DuckTransaction::Commit(AttachedDatabase &db, transaction_t commit_id, bool checkpoint) noexcept { +ErrorData DuckTransaction::Commit(AttachedDatabase &db, transaction_t commit_id, bool checkpoint) noexcept { // "checkpoint" parameter indicates if the caller will checkpoint. If checkpoint == // true: Then this function will NOT write to the WAL or flush/persist. // This method only makes commit in memory, expecting caller to checkpoint/flush. @@ -129,6 +129,7 @@ string DuckTransaction::Commit(AttachedDatabase &db, transaction_t commit_id, bo } else { log = nullptr; } + try { storage->Commit(commit_state, *this); undo_buffer.Commit(iterator_state, log, commit_id); @@ -141,10 +142,10 @@ string DuckTransaction::Commit(AttachedDatabase &db, transaction_t commit_id, bo if (storage_commit_state) { storage_commit_state->FlushCommit(); } - return string(); + return ErrorData(); } catch (std::exception &ex) { undo_buffer.RevertCommit(iterator_state, this->transaction_id); - return ex.what(); + return ErrorData(ex); } } diff --git a/src/duckdb/src/transaction/duck_transaction_manager.cpp b/src/duckdb/src/transaction/duck_transaction_manager.cpp index b16be4c98..e55a72653 100644 --- a/src/duckdb/src/transaction/duck_transaction_manager.cpp +++ b/src/duckdb/src/transaction/duck_transaction_manager.cpp @@ -1,6 +1,7 @@ #include "duckdb/transaction/duck_transaction_manager.hpp" #include "duckdb/catalog/catalog_set.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/common/helper.hpp" #include "duckdb/common/types/timestamp.hpp" @@ -166,7 +167,7 @@ bool DuckTransactionManager::CanCheckpoint(optional_ptr current return true; } -string DuckTransactionManager::CommitTransaction(ClientContext &context, Transaction &transaction_p) { +ErrorData DuckTransactionManager::CommitTransaction(ClientContext &context, Transaction &transaction_p) { auto &transaction = transaction_p.Cast(); vector client_locks; auto lock = make_uniq>(transaction_lock); @@ -183,8 +184,8 @@ string DuckTransactionManager::CommitTransaction(ClientContext &context, Transac // obtain a commit id for the transaction transaction_t commit_id = current_start_timestamp++; // commit the UndoBuffer of the transaction - string error = transaction.Commit(db, commit_id, checkpoint); - if (!error.empty()) { + auto error = transaction.Commit(db, commit_id, checkpoint); + if (error.HasError()) { // commit unsuccessful: rollback the transaction instead checkpoint = false; transaction.commit_id = 0; diff --git a/src/duckdb/src/transaction/meta_transaction.cpp b/src/duckdb/src/transaction/meta_transaction.cpp index e70491180..739d59d59 100644 --- a/src/duckdb/src/transaction/meta_transaction.cpp +++ b/src/duckdb/src/transaction/meta_transaction.cpp @@ -2,6 +2,7 @@ #include "duckdb/main/client_context.hpp" #include "duckdb/main/attached_database.hpp" #include "duckdb/transaction/transaction_manager.hpp" +#include "duckdb/common/exception/transaction_exception.hpp" namespace duckdb { @@ -57,8 +58,8 @@ Transaction &Transaction::Get(ClientContext &context, Catalog &catalog) { return Transaction::Get(context, catalog.GetAttached()); } -string MetaTransaction::Commit() { - string error; +ErrorData MetaTransaction::Commit() { + ErrorData error; // commit transactions in reverse order for (idx_t i = all_transactions.size(); i > 0; i--) { auto &db = all_transactions[i - 1].get(); @@ -68,7 +69,7 @@ string MetaTransaction::Commit() { } auto &transaction_manager = db.GetTransactionManager(); auto &transaction = entry->second.get(); - if (error.empty()) { + if (!error.HasError()) { // commit error = transaction_manager.CommitTransaction(context, transaction); } else { diff --git a/src/duckdb/src/transaction/transaction_context.cpp b/src/duckdb/src/transaction/transaction_context.cpp index 4b65a721b..22c8b85ba 100644 --- a/src/duckdb/src/transaction/transaction_context.cpp +++ b/src/duckdb/src/transaction/transaction_context.cpp @@ -1,5 +1,5 @@ #include "duckdb/transaction/transaction_context.hpp" - +#include "duckdb/common/exception/transaction_exception.hpp" #include "duckdb/common/exception.hpp" #include "duckdb/transaction/meta_transaction.hpp" #include "duckdb/transaction/transaction_manager.hpp" @@ -45,9 +45,9 @@ void TransactionContext::Commit() { } auto transaction = std::move(current_transaction); ClearTransaction(); - string error = transaction->Commit(); - if (!error.empty()) { - throw TransactionException("Failed to commit: %s", error); + auto error = transaction->Commit(); + if (error.HasError()) { + throw TransactionException("Failed to commit: %s", error.RawMessage()); } } diff --git a/src/duckdb/src/verification/fetch_row_verifier.cpp b/src/duckdb/src/verification/fetch_row_verifier.cpp new file mode 100644 index 000000000..a3be8111c --- /dev/null +++ b/src/duckdb/src/verification/fetch_row_verifier.cpp @@ -0,0 +1,13 @@ +#include "duckdb/verification/fetch_row_verifier.hpp" + +namespace duckdb { + +FetchRowVerifier::FetchRowVerifier(unique_ptr statement_p) + : StatementVerifier(VerificationType::FETCH_ROW_AS_SCAN, "FetchRow as Scan", std::move(statement_p)) { +} + +unique_ptr FetchRowVerifier::Create(const SQLStatement &statement_p) { + return make_uniq(statement_p.Copy()); +} + +} // namespace duckdb diff --git a/src/duckdb/src/verification/prepared_statement_verifier.cpp b/src/duckdb/src/verification/prepared_statement_verifier.cpp index 31a1d9ffd..f3707e165 100644 --- a/src/duckdb/src/verification/prepared_statement_verifier.cpp +++ b/src/duckdb/src/verification/prepared_statement_verifier.cpp @@ -1,6 +1,6 @@ #include "duckdb/verification/prepared_statement_verifier.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/parser/expression/parameter_expression.hpp" #include "duckdb/parser/parsed_expression_iterator.hpp" #include "duckdb/parser/statement/drop_statement.hpp" @@ -93,14 +93,12 @@ bool PreparedStatementVerifier::Run( execute_result->ThrowError("Failed execute during verify: "); } materialized_result = unique_ptr_cast(std::move(execute_result)); - } catch (const Exception &ex) { - if (ex.type != ExceptionType::PARAMETER_NOT_ALLOWED) { - materialized_result = make_uniq(PreservedError(ex)); + } catch (const std::exception &ex) { + ErrorData error(ex); + if (error.Type() != ExceptionType::PARAMETER_NOT_ALLOWED) { + materialized_result = make_uniq(std::move(error)); } failed = true; - } catch (std::exception &ex) { - materialized_result = make_uniq(PreservedError(ex)); - failed = true; } run(string(), std::move(dealloc_statement)); context.interrupted = false; diff --git a/src/duckdb/src/verification/statement_verifier.cpp b/src/duckdb/src/verification/statement_verifier.cpp index 3d473302d..c3c971588 100644 --- a/src/duckdb/src/verification/statement_verifier.cpp +++ b/src/duckdb/src/verification/statement_verifier.cpp @@ -1,6 +1,6 @@ #include "duckdb/verification/statement_verifier.hpp" -#include "duckdb/common/preserved_error.hpp" +#include "duckdb/common/error_data.hpp" #include "duckdb/common/types/column/column_data_collection.hpp" #include "duckdb/parser/parser.hpp" #include "duckdb/verification/copied_statement_verifier.hpp" @@ -10,6 +10,7 @@ #include "duckdb/verification/prepared_statement_verifier.hpp" #include "duckdb/verification/unoptimized_statement_verifier.hpp" #include "duckdb/verification/no_operator_caching_verifier.hpp" +#include "duckdb/verification/fetch_row_verifier.hpp" namespace duckdb { @@ -42,6 +43,8 @@ unique_ptr StatementVerifier::Create(VerificationType type, c return PreparedStatementVerifier::Create(statement_p); case VerificationType::EXTERNAL: return ExternalStatementVerifier::Create(statement_p); + case VerificationType::FETCH_ROW_AS_SCAN: + return FetchRowVerifier::Create(statement_p); case VerificationType::INVALID: default: throw InternalException("Invalid statement verification type!"); @@ -108,18 +111,16 @@ bool StatementVerifier::Run( context.config.enable_optimizer = !DisableOptimizer(); context.config.enable_caching_operators = !DisableOperatorCaching(); context.config.force_external = ForceExternal(); + context.config.force_fetch_row = ForceFetchRow(); try { auto result = run(query, std::move(statement)); if (result->HasError()) { failed = true; } materialized_result = unique_ptr_cast(std::move(result)); - } catch (const Exception &ex) { - failed = true; - materialized_result = make_uniq(PreservedError(ex)); } catch (std::exception &ex) { failed = true; - materialized_result = make_uniq(PreservedError(ex)); + materialized_result = make_uniq(ErrorData(ex)); } context.interrupted = false; diff --git a/src/duckdb/third_party/fast_float/fast_float/fast_float.h b/src/duckdb/third_party/fast_float/fast_float/fast_float.h index d072fc756..34e627b21 100644 --- a/src/duckdb/third_party/fast_float/fast_float/fast_float.h +++ b/src/duckdb/third_party/fast_float/fast_float/fast_float.h @@ -522,33 +522,61 @@ parsed_number_string parse_number_string(const char *p, const char *pend, const uint64_t i = 0; // an unsigned int avoids signed overflows (which are bad) - while ((p != pend) && is_integer(*p)) { - // a multiplication by 10 is cheaper than an arbitrary integer - // multiplication - i = 10 * i + - uint64_t(*p - '0'); // might overflow, we will handle the overflow later - ++p; + while ((p != pend)) { + if(is_integer(*p)) { + // a multiplication by 10 is cheaper than an arbitrary integer + // multiplication + i = 10 * i + + uint64_t(*p - '0'); // might overflow, we will handle the overflow later + ++p; + if(p != pend && *p == '_') { + // skip 1 underscore if it is not the last character and followed by a digit + ++p; + if(p == pend || !is_integer(*p)) { + return answer; + } + } + } + else { + break; + } } const char *const end_of_integer_part = p; int64_t digit_count = int64_t(end_of_integer_part - start_digits); int64_t exponent = 0; if ((p != pend) && (*p == decimal_separator)) { ++p; - // Fast approach only tested under little endian systems - if ((p + 8 <= pend) && is_made_of_eight_digits_fast(p)) { - i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok - p += 8; + + // Fast approach only tested under little endian systems if ((p + 8 <= pend) && is_made_of_eight_digits_fast(p)) { i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok p += 8; + if ((p + 8 <= pend) && is_made_of_eight_digits_fast(p)) { + i = i * 100000000 + parse_eight_digits_unrolled(p); // in rare cases, this will overflow, but that's ok + p += 8; + } } - } - while ((p != pend) && is_integer(*p)) { - uint8_t digit = uint8_t(*p - '0'); - ++p; - i = i * 10 + digit; // in rare cases, this will overflow, but that's ok + + int64_t skipped_underscores = 0; + while ((p != pend)) { + if(is_integer(*p)) { + uint8_t digit = uint8_t(*p - '0'); + ++p; + i = i * 10 + digit; // in rare cases, this will overflow, but that's ok + + if(p != pend && *p == '_') { + // skip 1 underscore if it is not the last character and followed by a digit + ++p; + ++skipped_underscores; + if(p == pend || !is_integer(*p)) { + return answer; + } + } + } else { + break; + } } - exponent = end_of_integer_part + 1 - p; + exponent = end_of_integer_part + 1 - p + skipped_underscores; digit_count -= exponent; } // we must have encountered at least one integer! @@ -574,12 +602,25 @@ parsed_number_string parse_number_string(const char *p, const char *pend, const // Otherwise, we will be ignoring the 'e'. p = location_of_e; } else { - while ((p != pend) && is_integer(*p)) { - uint8_t digit = uint8_t(*p - '0'); - if (exp_number < 0x10000) { - exp_number = 10 * exp_number + digit; + while ((p != pend)) { + if(is_integer(*p)) { + uint8_t digit = uint8_t(*p - '0'); + if (exp_number < 0x10000) { + exp_number = 10 * exp_number + digit; + } + ++p; + + if(p != pend && *p == '_') { + // skip 1 underscore if it is not the last character and followed by a digit + ++p; + if(p == pend || !is_integer(*p)) { + return answer; + } + } + } + else { + break; } - ++p; } if(neg_exp) { exp_number = - exp_number; } exponent += exp_number; @@ -612,20 +653,50 @@ parsed_number_string parse_number_string(const char *p, const char *pend, const i = 0; p = start_digits; const uint64_t minimal_nineteen_digit_integer{1000000000000000000}; - while((i < minimal_nineteen_digit_integer) && (p != pend) && is_integer(*p)) { - i = i * 10 + uint64_t(*p - '0'); - ++p; + while((i < minimal_nineteen_digit_integer) && (p != pend)) { + if (is_integer(*p)){ + i = i * 10 + uint64_t(*p - '0'); + ++p; + + if(p != pend && *p == '_') { + // skip 1 underscore if it is not the last character and followed by a digit + ++p; + if(p == pend || !is_integer(*p)) { + answer.valid = false; + return answer; + } + } + } + else { + break; + } } if (i >= minimal_nineteen_digit_integer) { // We have a big integers exponent = end_of_integer_part - p + exp_number; } else { // We have a value with a fractional component. p++; // skip the decimal_separator const char *first_after_period = p; - while((i < minimal_nineteen_digit_integer) && (p != pend) && is_integer(*p)) { - i = i * 10 + uint64_t(*p - '0'); - ++p; + int64_t skipped_underscores = 0; + while((i < minimal_nineteen_digit_integer) && (p != pend)) { + if(is_integer(*p)) { + i = i * 10 + uint64_t(*p - '0'); + ++p; + + if(p != pend && *p == '_') { + // skip 1 underscore if it is not the last character and followed by a digit + ++p; + ++skipped_underscores; + if(p == pend || !is_integer(*p)) { + answer.valid = false; + return answer; + } + } + } + else { + break; + } } - exponent = first_after_period - p + exp_number; + exponent = first_after_period - p + exp_number + skipped_underscores; } // We have now corrected both exponent and i, to a truncated value } diff --git a/src/duckdb/third_party/fmt/include/fmt/format-inl.h b/src/duckdb/third_party/fmt/include/fmt/format-inl.h index cef5359cf..1755025ae 100644 --- a/src/duckdb/third_party/fmt/include/fmt/format-inl.h +++ b/src/duckdb/third_party/fmt/include/fmt/format-inl.h @@ -1173,7 +1173,7 @@ template <> struct formatter { }; FMT_FUNC void internal::error_handler::on_error(std::string message) { - FMT_THROW(duckdb::Exception(message)); + FMT_THROW(duckdb::InvalidInputException(message)); } FMT_END_NAMESPACE diff --git a/src/duckdb/third_party/fmt/include/fmt/format.h b/src/duckdb/third_party/fmt/include/fmt/format.h index ced7d1460..3bf2f6063 100644 --- a/src/duckdb/third_party/fmt/include/fmt/format.h +++ b/src/duckdb/third_party/fmt/include/fmt/format.h @@ -1535,7 +1535,7 @@ template class basic_writer { } FMT_NORETURN void on_error(std::string error) { - FMT_THROW(duckdb::Exception(error)); + FMT_THROW(duckdb::InvalidInputException(error)); } }; @@ -1768,7 +1768,7 @@ class arg_formatter_base { void write(const char_type* value) { if (!value) { - FMT_THROW(duckdb::Exception("string pointer is null")); + FMT_THROW(duckdb::InternalException("string pointer is null")); } else { auto length = std::char_traits::length(value); basic_string_view sv(value, length); diff --git a/src/duckdb/third_party/fmt/include/fmt/printf.h b/src/duckdb/third_party/fmt/include/fmt/printf.h index 014975dfa..6617d9d49 100644 --- a/src/duckdb/third_party/fmt/include/fmt/printf.h +++ b/src/duckdb/third_party/fmt/include/fmt/printf.h @@ -43,13 +43,13 @@ class printf_precision_handler { template ::value)> int operator()(T value) { if (!int_checker::is_signed>::fits_in_int(value)) - FMT_THROW(duckdb::Exception("number is too big")); + FMT_THROW(duckdb::InvalidInputException("number is too big")); return (std::max)(static_cast(value), 0); } template ::value)> int operator()(T) { - FMT_THROW(duckdb::Exception("precision is not integer")); + FMT_THROW(duckdb::InvalidInputException("precision is not integer")); return 0; } }; @@ -164,13 +164,13 @@ template class printf_width_handler { width = 0 - width; } unsigned int_max = max_value(); - if (width > int_max) FMT_THROW(duckdb::Exception("number is too big")); + if (width > int_max) FMT_THROW(duckdb::InvalidInputException("number is too big")); return static_cast(width); } template ::value)> unsigned operator()(T) { - FMT_THROW(duckdb::Exception("width is not integer")); + FMT_THROW(duckdb::InvalidInputException("width is not integer")); return 0; } }; @@ -550,7 +550,7 @@ OutputIt basic_printf_context::format() { } // Parse type. - if (it == end) FMT_THROW(duckdb::Exception("invalid format string")); + if (it == end) FMT_THROW(duckdb::InvalidInputException("invalid format string")); specs.type = static_cast(*it++); if (arg.is_integral()) { // Normalize type. diff --git a/src/duckdb/third_party/libpg_query/include/nodes/nodes.hpp b/src/duckdb/third_party/libpg_query/include/nodes/nodes.hpp index 0f047e442..32c08a874 100644 --- a/src/duckdb/third_party/libpg_query/include/nodes/nodes.hpp +++ b/src/duckdb/third_party/libpg_query/include/nodes/nodes.hpp @@ -329,6 +329,7 @@ typedef enum PGNodeTag { T_PGDropStmt, T_PGTruncateStmt, T_PGCommentStmt, + T_PGCommentOnStmt, T_PGFetchStmt, T_PGIndexStmt, T_PGCreateFunctionStmt, @@ -364,6 +365,7 @@ typedef enum PGNodeTag { T_PGReindexStmt, T_PGCheckPointStmt, T_PGCreateSchemaStmt, + T_PGCreateSecretStmt, T_PGAlterDatabaseStmt, T_PGAlterDatabaseSetStmt, T_PGAlterRoleSetStmt, @@ -378,6 +380,7 @@ typedef enum PGNodeTag { T_PGDeallocateStmt, T_PGDeclareCursorStmt, T_PGCreateTableSpaceStmt, + T_PGDropSecretStmt, T_PGDropTableSpaceStmt, T_PGAlterObjectDependsStmt, T_PGAlterObjectSchemaStmt, 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 a1ffda302..b26624cfc 100644 --- a/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp +++ b/src/duckdb/third_party/libpg_query/include/nodes/parsenodes.hpp @@ -2163,4 +2163,43 @@ typedef struct PGUseStmt { } PGUseStmt; +/* ---------------------- + * Create Secret Statement + * ---------------------- + */ +typedef struct PGCreateSecretStmt { + PGNodeTag type; + char *persist_type; /* the requested persist mode */ + char *secret_name; /* name of the secret */ + char *secret_storage; /* the optional storage type of the secret */ + PGList *scope; /* optionally the scopes of the secret */ + PGList *options; /* Secret options */ + PGOnCreateConflict onconflict; /* what to do on create conflict */ +} PGCreateSecretStmt; + + +/* ---------------------- + * Drop Secret Statement + * ---------------------- + */ +typedef struct PGDropSecretStmt { + PGNodeTag type; + char *persist_type; /* the requested persist mode */ + char *secret_name; /* name of the secret */ + char *secret_storage; + bool missing_ok; +} PGDropSecretStmt; + +/* ---------------------- + * Comment On Statement + * ---------------------- + */ +typedef struct PGCommentOnStmt { + PGNodeTag type; + PGObjectType object_type; /* object type */ + PGRangeVar *name; /* the object to comment on */ + PGNode *value; /* the comment: a string or NULL*/ + PGNode *column_expr; +} PGCommentOnStmt; + } 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 7fa232145..0154683a7 100644 --- a/src/duckdb/third_party/libpg_query/include/parser/gram.hpp +++ b/src/duckdb/third_party/libpg_query/include/parser/gram.hpp @@ -1,14 +1,14 @@ -/* A Bison parser, made by GNU Bison 2.3. */ +/* A Bison parser, made by GNU Bison 3.8.2. */ -/* Skeleton interface for Bison's Yacc-like parsers in C +/* Bison interface for Yacc-like parsers in C - Copyright (C) 1984, 1989, 1990, 2000, 2001, 2002, 2003, 2004, 2005, 2006 - Free Software Foundation, Inc. + Copyright (C) 1984, 1989-1990, 2000-2015, 2018-2021 Free Software Foundation, + Inc. - This program is free software; you can redistribute it and/or modify + This program is free software: you can redistribute it and/or modify it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2, or (at your option) - any later version. + the Free Software Foundation, either version 3 of the License, or + (at your option) any later version. This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of @@ -16,9 +16,7 @@ GNU General Public License for more details. You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, - Boston, MA 02110-1301, USA. */ + along with this program. If not, see . */ /* As a special exception, you may create a larger work that contains part or all of the Bison parser skeleton and distribute that work @@ -33,1017 +31,539 @@ This special exception was added by the Free Software Foundation in version 2.2 of Bison. */ -/* Tokens. */ +/* DO NOT RELY ON FEATURES THAT ARE NOT DOCUMENTED in the manual, + especially those whose name start with YY_ or yy_. They are + private implementation details that can be changed or removed. */ + +#ifndef YY_BASE_YY_THIRD_PARTY_LIBPG_QUERY_GRAMMAR_GRAMMAR_OUT_HPP_INCLUDED +# define YY_BASE_YY_THIRD_PARTY_LIBPG_QUERY_GRAMMAR_GRAMMAR_OUT_HPP_INCLUDED +/* Debug traces. */ +#ifndef YYDEBUG +# define YYDEBUG 0 +#endif +#if YYDEBUG +extern int base_yydebug; +#endif + +/* Token kinds. */ #ifndef YYTOKENTYPE # define YYTOKENTYPE - /* Put the tokens into the symbol table, so that GDB and other debuggers - know about them. */ - enum yytokentype { - IDENT = 258, - FCONST = 259, - SCONST = 260, - BCONST = 261, - XCONST = 262, - Op = 263, - ICONST = 264, - PARAM = 265, - TYPECAST = 266, - DOT_DOT = 267, - COLON_EQUALS = 268, - EQUALS_GREATER = 269, - INTEGER_DIVISION = 270, - POWER_OF = 271, - LAMBDA_ARROW = 272, - DOUBLE_ARROW = 273, - LESS_EQUALS = 274, - GREATER_EQUALS = 275, - NOT_EQUALS = 276, - ABORT_P = 277, - ABSOLUTE_P = 278, - ACCESS = 279, - ACTION = 280, - ADD_P = 281, - ADMIN = 282, - AFTER = 283, - AGGREGATE = 284, - ALL = 285, - ALSO = 286, - ALTER = 287, - ALWAYS = 288, - ANALYSE = 289, - ANALYZE = 290, - AND = 291, - ANTI = 292, - ANY = 293, - ARRAY = 294, - AS = 295, - ASC_P = 296, - ASOF = 297, - ASSERTION = 298, - ASSIGNMENT = 299, - ASYMMETRIC = 300, - AT = 301, - ATTACH = 302, - ATTRIBUTE = 303, - AUTHORIZATION = 304, - BACKWARD = 305, - BEFORE = 306, - BEGIN_P = 307, - BETWEEN = 308, - BIGINT = 309, - BINARY = 310, - BIT = 311, - BOOLEAN_P = 312, - BOTH = 313, - BY = 314, - CACHE = 315, - CALL_P = 316, - CALLED = 317, - CASCADE = 318, - CASCADED = 319, - CASE = 320, - CAST = 321, - CATALOG_P = 322, - CENTURIES_P = 323, - CENTURY_P = 324, - CHAIN = 325, - CHAR_P = 326, - CHARACTER = 327, - CHARACTERISTICS = 328, - CHECK_P = 329, - CHECKPOINT = 330, - CLASS = 331, - CLOSE = 332, - CLUSTER = 333, - COALESCE = 334, - COLLATE = 335, - COLLATION = 336, - COLUMN = 337, - COLUMNS = 338, - COMMENT = 339, - COMMENTS = 340, - COMMIT = 341, - COMMITTED = 342, - COMPRESSION = 343, - CONCURRENTLY = 344, - CONFIGURATION = 345, - CONFLICT = 346, - CONNECTION = 347, - CONSTRAINT = 348, - CONSTRAINTS = 349, - CONTENT_P = 350, - CONTINUE_P = 351, - CONVERSION_P = 352, - COPY = 353, - COST = 354, - CREATE_P = 355, - CROSS = 356, - CSV = 357, - CUBE = 358, - CURRENT_P = 359, - CURSOR = 360, - CYCLE = 361, - DATA_P = 362, - DATABASE = 363, - DAY_P = 364, - DAYS_P = 365, - DEALLOCATE = 366, - DEC = 367, - DECADE_P = 368, - DECADES_P = 369, - DECIMAL_P = 370, - DECLARE = 371, - DEFAULT = 372, - DEFAULTS = 373, - DEFERRABLE = 374, - DEFERRED = 375, - DEFINER = 376, - DELETE_P = 377, - DELIMITER = 378, - DELIMITERS = 379, - DEPENDS = 380, - DESC_P = 381, - DESCRIBE = 382, - DETACH = 383, - DICTIONARY = 384, - DISABLE_P = 385, - DISCARD = 386, - DISTINCT = 387, - DO = 388, - DOCUMENT_P = 389, - DOMAIN_P = 390, - DOUBLE_P = 391, - DROP = 392, - EACH = 393, - ELSE = 394, - ENABLE_P = 395, - ENCODING = 396, - ENCRYPTED = 397, - END_P = 398, - ENUM_P = 399, - ESCAPE = 400, - EVENT = 401, - EXCEPT = 402, - EXCLUDE = 403, - EXCLUDING = 404, - EXCLUSIVE = 405, - EXECUTE = 406, - EXISTS = 407, - EXPLAIN = 408, - EXPORT_P = 409, - EXPORT_STATE = 410, - EXTENSION = 411, - EXTERNAL = 412, - EXTRACT = 413, - FALSE_P = 414, - FAMILY = 415, - FETCH = 416, - FILTER = 417, - FIRST_P = 418, - FLOAT_P = 419, - FOLLOWING = 420, - FOR = 421, - FORCE = 422, - FOREIGN = 423, - FORWARD = 424, - FREEZE = 425, - FROM = 426, - FULL = 427, - FUNCTION = 428, - FUNCTIONS = 429, - GENERATED = 430, - GLOB = 431, - GLOBAL = 432, - GRANT = 433, - GRANTED = 434, - GROUP_P = 435, - GROUPING = 436, - GROUPING_ID = 437, - GROUPS = 438, - HANDLER = 439, - HAVING = 440, - HEADER_P = 441, - HOLD = 442, - HOUR_P = 443, - HOURS_P = 444, - IDENTITY_P = 445, - IF_P = 446, - IGNORE_P = 447, - ILIKE = 448, - IMMEDIATE = 449, - IMMUTABLE = 450, - IMPLICIT_P = 451, - IMPORT_P = 452, - IN_P = 453, - INCLUDE_P = 454, - INCLUDING = 455, - INCREMENT = 456, - INDEX = 457, - INDEXES = 458, - INHERIT = 459, - INHERITS = 460, - INITIALLY = 461, - INLINE_P = 462, - INNER_P = 463, - INOUT = 464, - INPUT_P = 465, - INSENSITIVE = 466, - INSERT = 467, - INSTALL = 468, - INSTEAD = 469, - INT_P = 470, - INTEGER = 471, - INTERSECT = 472, - INTERVAL = 473, - INTO = 474, - INVOKER = 475, - IS = 476, - ISNULL = 477, - ISOLATION = 478, - JOIN = 479, - JSON = 480, - KEY = 481, - LABEL = 482, - LANGUAGE = 483, - LARGE_P = 484, - LAST_P = 485, - LATERAL_P = 486, - LEADING = 487, - LEAKPROOF = 488, - LEFT = 489, - LEVEL = 490, - LIKE = 491, - LIMIT = 492, - LISTEN = 493, - LOAD = 494, - LOCAL = 495, - LOCATION = 496, - LOCK_P = 497, - LOCKED = 498, - LOGGED = 499, - MACRO = 500, - MAP = 501, - MAPPING = 502, - MATCH = 503, - MATERIALIZED = 504, - MAXVALUE = 505, - METHOD = 506, - MICROSECOND_P = 507, - MICROSECONDS_P = 508, - MILLENNIA_P = 509, - MILLENNIUM_P = 510, - MILLISECOND_P = 511, - MILLISECONDS_P = 512, - MINUTE_P = 513, - MINUTES_P = 514, - MINVALUE = 515, - MODE = 516, - MONTH_P = 517, - MONTHS_P = 518, - MOVE = 519, - NAME_P = 520, - NAMES = 521, - NATIONAL = 522, - NATURAL = 523, - NCHAR = 524, - NEW = 525, - NEXT = 526, - NO = 527, - NONE = 528, - NOT = 529, - NOTHING = 530, - NOTIFY = 531, - NOTNULL = 532, - NOWAIT = 533, - NULL_P = 534, - NULLIF = 535, - NULLS_P = 536, - NUMERIC = 537, - OBJECT_P = 538, - OF = 539, - OFF = 540, - OFFSET = 541, - OIDS = 542, - OLD = 543, - ON = 544, - ONLY = 545, - OPERATOR = 546, - OPTION = 547, - OPTIONS = 548, - OR = 549, - ORDER = 550, - ORDINALITY = 551, - OTHERS = 552, - OUT_P = 553, - OUTER_P = 554, - OVER = 555, - OVERLAPS = 556, - OVERLAY = 557, - OVERRIDING = 558, - OWNED = 559, - OWNER = 560, - PARALLEL = 561, - PARSER = 562, - PARTIAL = 563, - PARTITION = 564, - PASSING = 565, - PASSWORD = 566, - PERCENT = 567, - PIVOT = 568, - PIVOT_LONGER = 569, - PIVOT_WIDER = 570, - PLACING = 571, - PLANS = 572, - POLICY = 573, - POSITION = 574, - POSITIONAL = 575, - PRAGMA_P = 576, - PRECEDING = 577, - PRECISION = 578, - PREPARE = 579, - PREPARED = 580, - PRESERVE = 581, - PRIMARY = 582, - PRIOR = 583, - PRIVILEGES = 584, - PROCEDURAL = 585, - PROCEDURE = 586, - PROGRAM = 587, - PUBLICATION = 588, - QUALIFY = 589, - QUOTE = 590, - RANGE = 591, - READ_P = 592, - REAL = 593, - REASSIGN = 594, - RECHECK = 595, - RECURSIVE = 596, - REF = 597, - REFERENCES = 598, - REFERENCING = 599, - REFRESH = 600, - REINDEX = 601, - RELATIVE_P = 602, - RELEASE = 603, - RENAME = 604, - REPEATABLE = 605, - REPLACE = 606, - REPLICA = 607, - RESET = 608, - RESPECT_P = 609, - RESTART = 610, - RESTRICT = 611, - RETURNING = 612, - RETURNS = 613, - REVOKE = 614, - RIGHT = 615, - ROLE = 616, - ROLLBACK = 617, - ROLLUP = 618, - ROW = 619, - ROWS = 620, - RULE = 621, - SAMPLE = 622, - SAVEPOINT = 623, - SCHEMA = 624, - SCHEMAS = 625, - SCROLL = 626, - SEARCH = 627, - SECOND_P = 628, - SECONDS_P = 629, - SECURITY = 630, - SELECT = 631, - SEMI = 632, - SEQUENCE = 633, - SEQUENCES = 634, - SERIALIZABLE = 635, - SERVER = 636, - SESSION = 637, - SET = 638, - SETOF = 639, - SETS = 640, - SHARE = 641, - SHOW = 642, - SIMILAR = 643, - SIMPLE = 644, - SKIP = 645, - SMALLINT = 646, - SNAPSHOT = 647, - SOME = 648, - SQL_P = 649, - STABLE = 650, - STANDALONE_P = 651, - START = 652, - STATEMENT = 653, - STATISTICS = 654, - STDIN = 655, - STDOUT = 656, - STORAGE = 657, - STORED = 658, - STRICT_P = 659, - STRIP_P = 660, - STRUCT = 661, - SUBSCRIPTION = 662, - SUBSTRING = 663, - SUMMARIZE = 664, - SYMMETRIC = 665, - SYSID = 666, - SYSTEM_P = 667, - TABLE = 668, - TABLES = 669, - TABLESAMPLE = 670, - TABLESPACE = 671, - TEMP = 672, - TEMPLATE = 673, - TEMPORARY = 674, - TEXT_P = 675, - THEN = 676, - TIES = 677, - TIME = 678, - TIMESTAMP = 679, - TO = 680, - TRAILING = 681, - TRANSACTION = 682, - TRANSFORM = 683, - TREAT = 684, - TRIGGER = 685, - TRIM = 686, - TRUE_P = 687, - TRUNCATE = 688, - TRUSTED = 689, - TRY_CAST = 690, - TYPE_P = 691, - TYPES_P = 692, - UNBOUNDED = 693, - UNCOMMITTED = 694, - UNENCRYPTED = 695, - UNION = 696, - UNIQUE = 697, - UNKNOWN = 698, - UNLISTEN = 699, - UNLOGGED = 700, - UNPIVOT = 701, - UNTIL = 702, - UPDATE = 703, - USE_P = 704, - USER = 705, - USING = 706, - VACUUM = 707, - VALID = 708, - VALIDATE = 709, - VALIDATOR = 710, - VALUE_P = 711, - VALUES = 712, - VARCHAR = 713, - VARIADIC = 714, - VARYING = 715, - VERBOSE = 716, - VERSION_P = 717, - VIEW = 718, - VIEWS = 719, - VIRTUAL = 720, - VOLATILE = 721, - WEEK_P = 722, - WEEKS_P = 723, - WHEN = 724, - WHERE = 725, - WHITESPACE_P = 726, - WINDOW = 727, - WITH = 728, - WITHIN = 729, - WITHOUT = 730, - WORK = 731, - WRAPPER = 732, - WRITE_P = 733, - XML_P = 734, - XMLATTRIBUTES = 735, - XMLCONCAT = 736, - XMLELEMENT = 737, - XMLEXISTS = 738, - XMLFOREST = 739, - XMLNAMESPACES = 740, - XMLPARSE = 741, - XMLPI = 742, - XMLROOT = 743, - XMLSERIALIZE = 744, - XMLTABLE = 745, - YEAR_P = 746, - YEARS_P = 747, - YES_P = 748, - ZONE = 749, - NOT_LA = 750, - NULLS_LA = 751, - WITH_LA = 752, - POSTFIXOP = 753, - UMINUS = 754 - }; + enum yytokentype + { + YYEMPTY = -2, + YYEOF = 0, /* "end of file" */ + YYerror = 256, /* error */ + YYUNDEF = 257, /* "invalid token" */ + IDENT = 258, /* IDENT */ + FCONST = 259, /* FCONST */ + SCONST = 260, /* SCONST */ + BCONST = 261, /* BCONST */ + XCONST = 262, /* XCONST */ + Op = 263, /* Op */ + ICONST = 264, /* ICONST */ + PARAM = 265, /* PARAM */ + TYPECAST = 266, /* TYPECAST */ + DOT_DOT = 267, /* DOT_DOT */ + COLON_EQUALS = 268, /* COLON_EQUALS */ + EQUALS_GREATER = 269, /* EQUALS_GREATER */ + INTEGER_DIVISION = 270, /* INTEGER_DIVISION */ + POWER_OF = 271, /* POWER_OF */ + LAMBDA_ARROW = 272, /* LAMBDA_ARROW */ + DOUBLE_ARROW = 273, /* DOUBLE_ARROW */ + LESS_EQUALS = 274, /* LESS_EQUALS */ + GREATER_EQUALS = 275, /* GREATER_EQUALS */ + NOT_EQUALS = 276, /* NOT_EQUALS */ + ABORT_P = 277, /* ABORT_P */ + ABSOLUTE_P = 278, /* ABSOLUTE_P */ + ACCESS = 279, /* ACCESS */ + ACTION = 280, /* ACTION */ + ADD_P = 281, /* ADD_P */ + ADMIN = 282, /* ADMIN */ + AFTER = 283, /* AFTER */ + AGGREGATE = 284, /* AGGREGATE */ + ALL = 285, /* ALL */ + ALSO = 286, /* ALSO */ + ALTER = 287, /* ALTER */ + ALWAYS = 288, /* ALWAYS */ + ANALYSE = 289, /* ANALYSE */ + ANALYZE = 290, /* ANALYZE */ + AND = 291, /* AND */ + ANTI = 292, /* ANTI */ + ANY = 293, /* ANY */ + ARRAY = 294, /* ARRAY */ + AS = 295, /* AS */ + ASC_P = 296, /* ASC_P */ + ASOF = 297, /* ASOF */ + ASSERTION = 298, /* ASSERTION */ + ASSIGNMENT = 299, /* ASSIGNMENT */ + ASYMMETRIC = 300, /* ASYMMETRIC */ + AT = 301, /* AT */ + ATTACH = 302, /* ATTACH */ + ATTRIBUTE = 303, /* ATTRIBUTE */ + AUTHORIZATION = 304, /* AUTHORIZATION */ + BACKWARD = 305, /* BACKWARD */ + BEFORE = 306, /* BEFORE */ + BEGIN_P = 307, /* BEGIN_P */ + BETWEEN = 308, /* BETWEEN */ + BIGINT = 309, /* BIGINT */ + BINARY = 310, /* BINARY */ + BIT = 311, /* BIT */ + BOOLEAN_P = 312, /* BOOLEAN_P */ + BOTH = 313, /* BOTH */ + BY = 314, /* BY */ + CACHE = 315, /* CACHE */ + CALL_P = 316, /* CALL_P */ + CALLED = 317, /* CALLED */ + CASCADE = 318, /* CASCADE */ + CASCADED = 319, /* CASCADED */ + CASE = 320, /* CASE */ + CAST = 321, /* CAST */ + CATALOG_P = 322, /* CATALOG_P */ + CENTURIES_P = 323, /* CENTURIES_P */ + CENTURY_P = 324, /* CENTURY_P */ + CHAIN = 325, /* CHAIN */ + CHAR_P = 326, /* CHAR_P */ + CHARACTER = 327, /* CHARACTER */ + CHARACTERISTICS = 328, /* CHARACTERISTICS */ + CHECK_P = 329, /* CHECK_P */ + CHECKPOINT = 330, /* CHECKPOINT */ + CLASS = 331, /* CLASS */ + CLOSE = 332, /* CLOSE */ + CLUSTER = 333, /* CLUSTER */ + COALESCE = 334, /* COALESCE */ + COLLATE = 335, /* COLLATE */ + COLLATION = 336, /* COLLATION */ + COLUMN = 337, /* COLUMN */ + COLUMNS = 338, /* COLUMNS */ + COMMENT = 339, /* COMMENT */ + COMMENTS = 340, /* COMMENTS */ + COMMIT = 341, /* COMMIT */ + COMMITTED = 342, /* COMMITTED */ + COMPRESSION = 343, /* COMPRESSION */ + CONCURRENTLY = 344, /* CONCURRENTLY */ + CONFIGURATION = 345, /* CONFIGURATION */ + CONFLICT = 346, /* CONFLICT */ + CONNECTION = 347, /* CONNECTION */ + CONSTRAINT = 348, /* CONSTRAINT */ + CONSTRAINTS = 349, /* CONSTRAINTS */ + CONTENT_P = 350, /* CONTENT_P */ + CONTINUE_P = 351, /* CONTINUE_P */ + CONVERSION_P = 352, /* CONVERSION_P */ + COPY = 353, /* COPY */ + COST = 354, /* COST */ + CREATE_P = 355, /* CREATE_P */ + CROSS = 356, /* CROSS */ + CSV = 357, /* CSV */ + CUBE = 358, /* CUBE */ + CURRENT_P = 359, /* CURRENT_P */ + CURSOR = 360, /* CURSOR */ + CYCLE = 361, /* CYCLE */ + DATA_P = 362, /* DATA_P */ + DATABASE = 363, /* DATABASE */ + DAY_P = 364, /* DAY_P */ + DAYS_P = 365, /* DAYS_P */ + DEALLOCATE = 366, /* DEALLOCATE */ + DEC = 367, /* DEC */ + DECADE_P = 368, /* DECADE_P */ + DECADES_P = 369, /* DECADES_P */ + DECIMAL_P = 370, /* DECIMAL_P */ + DECLARE = 371, /* DECLARE */ + DEFAULT = 372, /* DEFAULT */ + DEFAULTS = 373, /* DEFAULTS */ + DEFERRABLE = 374, /* DEFERRABLE */ + DEFERRED = 375, /* DEFERRED */ + DEFINER = 376, /* DEFINER */ + DELETE_P = 377, /* DELETE_P */ + DELIMITER = 378, /* DELIMITER */ + DELIMITERS = 379, /* DELIMITERS */ + DEPENDS = 380, /* DEPENDS */ + DESC_P = 381, /* DESC_P */ + DESCRIBE = 382, /* DESCRIBE */ + DETACH = 383, /* DETACH */ + DICTIONARY = 384, /* DICTIONARY */ + DISABLE_P = 385, /* DISABLE_P */ + DISCARD = 386, /* DISCARD */ + DISTINCT = 387, /* DISTINCT */ + DO = 388, /* DO */ + DOCUMENT_P = 389, /* DOCUMENT_P */ + DOMAIN_P = 390, /* DOMAIN_P */ + DOUBLE_P = 391, /* DOUBLE_P */ + DROP = 392, /* DROP */ + EACH = 393, /* EACH */ + ELSE = 394, /* ELSE */ + ENABLE_P = 395, /* ENABLE_P */ + ENCODING = 396, /* ENCODING */ + ENCRYPTED = 397, /* ENCRYPTED */ + END_P = 398, /* END_P */ + ENUM_P = 399, /* ENUM_P */ + ESCAPE = 400, /* ESCAPE */ + EVENT = 401, /* EVENT */ + EXCEPT = 402, /* EXCEPT */ + EXCLUDE = 403, /* EXCLUDE */ + EXCLUDING = 404, /* EXCLUDING */ + EXCLUSIVE = 405, /* EXCLUSIVE */ + EXECUTE = 406, /* EXECUTE */ + EXISTS = 407, /* EXISTS */ + EXPLAIN = 408, /* EXPLAIN */ + EXPORT_P = 409, /* EXPORT_P */ + EXPORT_STATE = 410, /* EXPORT_STATE */ + EXTENSION = 411, /* EXTENSION */ + EXTERNAL = 412, /* EXTERNAL */ + EXTRACT = 413, /* EXTRACT */ + FALSE_P = 414, /* FALSE_P */ + FAMILY = 415, /* FAMILY */ + FETCH = 416, /* FETCH */ + FILTER = 417, /* FILTER */ + FIRST_P = 418, /* FIRST_P */ + FLOAT_P = 419, /* FLOAT_P */ + FOLLOWING = 420, /* FOLLOWING */ + FOR = 421, /* FOR */ + FORCE = 422, /* FORCE */ + FOREIGN = 423, /* FOREIGN */ + FORWARD = 424, /* FORWARD */ + FREEZE = 425, /* FREEZE */ + FROM = 426, /* FROM */ + FULL = 427, /* FULL */ + FUNCTION = 428, /* FUNCTION */ + FUNCTIONS = 429, /* FUNCTIONS */ + GENERATED = 430, /* GENERATED */ + GLOB = 431, /* GLOB */ + GLOBAL = 432, /* GLOBAL */ + GRANT = 433, /* GRANT */ + GRANTED = 434, /* GRANTED */ + GROUP_P = 435, /* GROUP_P */ + GROUPING = 436, /* GROUPING */ + GROUPING_ID = 437, /* GROUPING_ID */ + GROUPS = 438, /* GROUPS */ + HANDLER = 439, /* HANDLER */ + HAVING = 440, /* HAVING */ + HEADER_P = 441, /* HEADER_P */ + HOLD = 442, /* HOLD */ + HOUR_P = 443, /* HOUR_P */ + HOURS_P = 444, /* HOURS_P */ + IDENTITY_P = 445, /* IDENTITY_P */ + IF_P = 446, /* IF_P */ + IGNORE_P = 447, /* IGNORE_P */ + ILIKE = 448, /* ILIKE */ + IMMEDIATE = 449, /* IMMEDIATE */ + IMMUTABLE = 450, /* IMMUTABLE */ + IMPLICIT_P = 451, /* IMPLICIT_P */ + IMPORT_P = 452, /* IMPORT_P */ + IN_P = 453, /* IN_P */ + INCLUDE_P = 454, /* INCLUDE_P */ + INCLUDING = 455, /* INCLUDING */ + INCREMENT = 456, /* INCREMENT */ + INDEX = 457, /* INDEX */ + INDEXES = 458, /* INDEXES */ + INHERIT = 459, /* INHERIT */ + INHERITS = 460, /* INHERITS */ + INITIALLY = 461, /* INITIALLY */ + INLINE_P = 462, /* INLINE_P */ + INNER_P = 463, /* INNER_P */ + INOUT = 464, /* INOUT */ + INPUT_P = 465, /* INPUT_P */ + INSENSITIVE = 466, /* INSENSITIVE */ + INSERT = 467, /* INSERT */ + INSTALL = 468, /* INSTALL */ + INSTEAD = 469, /* INSTEAD */ + INT_P = 470, /* INT_P */ + INTEGER = 471, /* INTEGER */ + INTERSECT = 472, /* INTERSECT */ + INTERVAL = 473, /* INTERVAL */ + INTO = 474, /* INTO */ + INVOKER = 475, /* INVOKER */ + IS = 476, /* IS */ + ISNULL = 477, /* ISNULL */ + ISOLATION = 478, /* ISOLATION */ + JOIN = 479, /* JOIN */ + JSON = 480, /* JSON */ + KEY = 481, /* KEY */ + LABEL = 482, /* LABEL */ + LANGUAGE = 483, /* LANGUAGE */ + LARGE_P = 484, /* LARGE_P */ + LAST_P = 485, /* LAST_P */ + LATERAL_P = 486, /* LATERAL_P */ + LEADING = 487, /* LEADING */ + LEAKPROOF = 488, /* LEAKPROOF */ + LEFT = 489, /* LEFT */ + LEVEL = 490, /* LEVEL */ + LIKE = 491, /* LIKE */ + LIMIT = 492, /* LIMIT */ + LISTEN = 493, /* LISTEN */ + LOAD = 494, /* LOAD */ + LOCAL = 495, /* LOCAL */ + LOCATION = 496, /* LOCATION */ + LOCK_P = 497, /* LOCK_P */ + LOCKED = 498, /* LOCKED */ + LOGGED = 499, /* LOGGED */ + MACRO = 500, /* MACRO */ + MAP = 501, /* MAP */ + MAPPING = 502, /* MAPPING */ + MATCH = 503, /* MATCH */ + MATERIALIZED = 504, /* MATERIALIZED */ + MAXVALUE = 505, /* MAXVALUE */ + METHOD = 506, /* METHOD */ + MICROSECOND_P = 507, /* MICROSECOND_P */ + MICROSECONDS_P = 508, /* MICROSECONDS_P */ + MILLENNIA_P = 509, /* MILLENNIA_P */ + MILLENNIUM_P = 510, /* MILLENNIUM_P */ + MILLISECOND_P = 511, /* MILLISECOND_P */ + MILLISECONDS_P = 512, /* MILLISECONDS_P */ + MINUTE_P = 513, /* MINUTE_P */ + MINUTES_P = 514, /* MINUTES_P */ + MINVALUE = 515, /* MINVALUE */ + MODE = 516, /* MODE */ + MONTH_P = 517, /* MONTH_P */ + MONTHS_P = 518, /* MONTHS_P */ + MOVE = 519, /* MOVE */ + NAME_P = 520, /* NAME_P */ + NAMES = 521, /* NAMES */ + NATIONAL = 522, /* NATIONAL */ + NATURAL = 523, /* NATURAL */ + NCHAR = 524, /* NCHAR */ + NEW = 525, /* NEW */ + NEXT = 526, /* NEXT */ + NO = 527, /* NO */ + NONE = 528, /* NONE */ + NOT = 529, /* NOT */ + NOTHING = 530, /* NOTHING */ + NOTIFY = 531, /* NOTIFY */ + NOTNULL = 532, /* NOTNULL */ + NOWAIT = 533, /* NOWAIT */ + NULL_P = 534, /* NULL_P */ + NULLIF = 535, /* NULLIF */ + NULLS_P = 536, /* NULLS_P */ + NUMERIC = 537, /* NUMERIC */ + OBJECT_P = 538, /* OBJECT_P */ + OF = 539, /* OF */ + OFF = 540, /* OFF */ + OFFSET = 541, /* OFFSET */ + OIDS = 542, /* OIDS */ + OLD = 543, /* OLD */ + ON = 544, /* ON */ + ONLY = 545, /* ONLY */ + OPERATOR = 546, /* OPERATOR */ + OPTION = 547, /* OPTION */ + OPTIONS = 548, /* OPTIONS */ + OR = 549, /* OR */ + ORDER = 550, /* ORDER */ + ORDINALITY = 551, /* ORDINALITY */ + OTHERS = 552, /* OTHERS */ + OUT_P = 553, /* OUT_P */ + OUTER_P = 554, /* OUTER_P */ + OVER = 555, /* OVER */ + OVERLAPS = 556, /* OVERLAPS */ + OVERLAY = 557, /* OVERLAY */ + OVERRIDING = 558, /* OVERRIDING */ + OWNED = 559, /* OWNED */ + OWNER = 560, /* OWNER */ + PARALLEL = 561, /* PARALLEL */ + PARSER = 562, /* PARSER */ + PARTIAL = 563, /* PARTIAL */ + PARTITION = 564, /* PARTITION */ + PASSING = 565, /* PASSING */ + PASSWORD = 566, /* PASSWORD */ + PERCENT = 567, /* PERCENT */ + PERSISTENT = 568, /* PERSISTENT */ + PIVOT = 569, /* PIVOT */ + PIVOT_LONGER = 570, /* PIVOT_LONGER */ + PIVOT_WIDER = 571, /* PIVOT_WIDER */ + PLACING = 572, /* PLACING */ + PLANS = 573, /* PLANS */ + POLICY = 574, /* POLICY */ + POSITION = 575, /* POSITION */ + POSITIONAL = 576, /* POSITIONAL */ + PRAGMA_P = 577, /* PRAGMA_P */ + PRECEDING = 578, /* PRECEDING */ + PRECISION = 579, /* PRECISION */ + PREPARE = 580, /* PREPARE */ + PREPARED = 581, /* PREPARED */ + PRESERVE = 582, /* PRESERVE */ + PRIMARY = 583, /* PRIMARY */ + PRIOR = 584, /* PRIOR */ + PRIVILEGES = 585, /* PRIVILEGES */ + PROCEDURAL = 586, /* PROCEDURAL */ + PROCEDURE = 587, /* PROCEDURE */ + PROGRAM = 588, /* PROGRAM */ + PUBLICATION = 589, /* PUBLICATION */ + QUALIFY = 590, /* QUALIFY */ + QUOTE = 591, /* QUOTE */ + RANGE = 592, /* RANGE */ + READ_P = 593, /* READ_P */ + REAL = 594, /* REAL */ + REASSIGN = 595, /* REASSIGN */ + RECHECK = 596, /* RECHECK */ + RECURSIVE = 597, /* RECURSIVE */ + REF = 598, /* REF */ + REFERENCES = 599, /* REFERENCES */ + REFERENCING = 600, /* REFERENCING */ + REFRESH = 601, /* REFRESH */ + REINDEX = 602, /* REINDEX */ + RELATIVE_P = 603, /* RELATIVE_P */ + RELEASE = 604, /* RELEASE */ + RENAME = 605, /* RENAME */ + REPEATABLE = 606, /* REPEATABLE */ + REPLACE = 607, /* REPLACE */ + REPLICA = 608, /* REPLICA */ + RESET = 609, /* RESET */ + RESPECT_P = 610, /* RESPECT_P */ + RESTART = 611, /* RESTART */ + RESTRICT = 612, /* RESTRICT */ + RETURNING = 613, /* RETURNING */ + RETURNS = 614, /* RETURNS */ + REVOKE = 615, /* REVOKE */ + RIGHT = 616, /* RIGHT */ + ROLE = 617, /* ROLE */ + ROLLBACK = 618, /* ROLLBACK */ + ROLLUP = 619, /* ROLLUP */ + ROW = 620, /* ROW */ + ROWS = 621, /* ROWS */ + RULE = 622, /* RULE */ + SAMPLE = 623, /* SAMPLE */ + SAVEPOINT = 624, /* SAVEPOINT */ + SCHEMA = 625, /* SCHEMA */ + SCHEMAS = 626, /* SCHEMAS */ + SCOPE = 627, /* SCOPE */ + SCROLL = 628, /* SCROLL */ + SEARCH = 629, /* SEARCH */ + SECOND_P = 630, /* SECOND_P */ + SECONDS_P = 631, /* SECONDS_P */ + SECRET = 632, /* SECRET */ + SECURITY = 633, /* SECURITY */ + SELECT = 634, /* SELECT */ + SEMI = 635, /* SEMI */ + SEQUENCE = 636, /* SEQUENCE */ + SEQUENCES = 637, /* SEQUENCES */ + SERIALIZABLE = 638, /* SERIALIZABLE */ + SERVER = 639, /* SERVER */ + SESSION = 640, /* SESSION */ + SET = 641, /* SET */ + SETOF = 642, /* SETOF */ + SETS = 643, /* SETS */ + SHARE = 644, /* SHARE */ + SHOW = 645, /* SHOW */ + SIMILAR = 646, /* SIMILAR */ + SIMPLE = 647, /* SIMPLE */ + SKIP = 648, /* SKIP */ + SMALLINT = 649, /* SMALLINT */ + SNAPSHOT = 650, /* SNAPSHOT */ + SOME = 651, /* SOME */ + SQL_P = 652, /* SQL_P */ + STABLE = 653, /* STABLE */ + STANDALONE_P = 654, /* STANDALONE_P */ + START = 655, /* START */ + STATEMENT = 656, /* STATEMENT */ + STATISTICS = 657, /* STATISTICS */ + STDIN = 658, /* STDIN */ + STDOUT = 659, /* STDOUT */ + STORAGE = 660, /* STORAGE */ + STORED = 661, /* STORED */ + STRICT_P = 662, /* STRICT_P */ + STRIP_P = 663, /* STRIP_P */ + STRUCT = 664, /* STRUCT */ + SUBSCRIPTION = 665, /* SUBSCRIPTION */ + SUBSTRING = 666, /* SUBSTRING */ + SUMMARIZE = 667, /* SUMMARIZE */ + SYMMETRIC = 668, /* SYMMETRIC */ + SYSID = 669, /* SYSID */ + SYSTEM_P = 670, /* SYSTEM_P */ + TABLE = 671, /* TABLE */ + TABLES = 672, /* TABLES */ + TABLESAMPLE = 673, /* TABLESAMPLE */ + TABLESPACE = 674, /* TABLESPACE */ + TEMP = 675, /* TEMP */ + TEMPLATE = 676, /* TEMPLATE */ + TEMPORARY = 677, /* TEMPORARY */ + TEXT_P = 678, /* TEXT_P */ + THEN = 679, /* THEN */ + TIES = 680, /* TIES */ + TIME = 681, /* TIME */ + TIMESTAMP = 682, /* TIMESTAMP */ + TO = 683, /* TO */ + TRAILING = 684, /* TRAILING */ + TRANSACTION = 685, /* TRANSACTION */ + TRANSFORM = 686, /* TRANSFORM */ + TREAT = 687, /* TREAT */ + TRIGGER = 688, /* TRIGGER */ + TRIM = 689, /* TRIM */ + TRUE_P = 690, /* TRUE_P */ + TRUNCATE = 691, /* TRUNCATE */ + TRUSTED = 692, /* TRUSTED */ + TRY_CAST = 693, /* TRY_CAST */ + TYPE_P = 694, /* TYPE_P */ + TYPES_P = 695, /* TYPES_P */ + UNBOUNDED = 696, /* UNBOUNDED */ + UNCOMMITTED = 697, /* UNCOMMITTED */ + UNENCRYPTED = 698, /* UNENCRYPTED */ + UNION = 699, /* UNION */ + UNIQUE = 700, /* UNIQUE */ + UNKNOWN = 701, /* UNKNOWN */ + UNLISTEN = 702, /* UNLISTEN */ + UNLOGGED = 703, /* UNLOGGED */ + UNPIVOT = 704, /* UNPIVOT */ + UNTIL = 705, /* UNTIL */ + UPDATE = 706, /* UPDATE */ + USE_P = 707, /* USE_P */ + USER = 708, /* USER */ + USING = 709, /* USING */ + VACUUM = 710, /* VACUUM */ + VALID = 711, /* VALID */ + VALIDATE = 712, /* VALIDATE */ + VALIDATOR = 713, /* VALIDATOR */ + VALUE_P = 714, /* VALUE_P */ + VALUES = 715, /* VALUES */ + VARCHAR = 716, /* VARCHAR */ + VARIADIC = 717, /* VARIADIC */ + VARYING = 718, /* VARYING */ + VERBOSE = 719, /* VERBOSE */ + VERSION_P = 720, /* VERSION_P */ + VIEW = 721, /* VIEW */ + VIEWS = 722, /* VIEWS */ + VIRTUAL = 723, /* VIRTUAL */ + VOLATILE = 724, /* VOLATILE */ + WEEK_P = 725, /* WEEK_P */ + WEEKS_P = 726, /* WEEKS_P */ + WHEN = 727, /* WHEN */ + WHERE = 728, /* WHERE */ + WHITESPACE_P = 729, /* WHITESPACE_P */ + WINDOW = 730, /* WINDOW */ + WITH = 731, /* WITH */ + WITHIN = 732, /* WITHIN */ + WITHOUT = 733, /* WITHOUT */ + WORK = 734, /* WORK */ + WRAPPER = 735, /* WRAPPER */ + WRITE_P = 736, /* WRITE_P */ + XML_P = 737, /* XML_P */ + XMLATTRIBUTES = 738, /* XMLATTRIBUTES */ + XMLCONCAT = 739, /* XMLCONCAT */ + XMLELEMENT = 740, /* XMLELEMENT */ + XMLEXISTS = 741, /* XMLEXISTS */ + XMLFOREST = 742, /* XMLFOREST */ + XMLNAMESPACES = 743, /* XMLNAMESPACES */ + XMLPARSE = 744, /* XMLPARSE */ + XMLPI = 745, /* XMLPI */ + XMLROOT = 746, /* XMLROOT */ + XMLSERIALIZE = 747, /* XMLSERIALIZE */ + XMLTABLE = 748, /* XMLTABLE */ + YEAR_P = 749, /* YEAR_P */ + YEARS_P = 750, /* YEARS_P */ + YES_P = 751, /* YES_P */ + ZONE = 752, /* ZONE */ + NOT_LA = 753, /* NOT_LA */ + NULLS_LA = 754, /* NULLS_LA */ + WITH_LA = 755, /* WITH_LA */ + POSTFIXOP = 756, /* POSTFIXOP */ + UMINUS = 757 /* UMINUS */ + }; + typedef enum yytokentype yytoken_kind_t; #endif -/* Tokens. */ -#define IDENT 258 -#define FCONST 259 -#define SCONST 260 -#define BCONST 261 -#define XCONST 262 -#define Op 263 -#define ICONST 264 -#define PARAM 265 -#define TYPECAST 266 -#define DOT_DOT 267 -#define COLON_EQUALS 268 -#define EQUALS_GREATER 269 -#define INTEGER_DIVISION 270 -#define POWER_OF 271 -#define LAMBDA_ARROW 272 -#define DOUBLE_ARROW 273 -#define LESS_EQUALS 274 -#define GREATER_EQUALS 275 -#define NOT_EQUALS 276 -#define ABORT_P 277 -#define ABSOLUTE_P 278 -#define ACCESS 279 -#define ACTION 280 -#define ADD_P 281 -#define ADMIN 282 -#define AFTER 283 -#define AGGREGATE 284 -#define ALL 285 -#define ALSO 286 -#define ALTER 287 -#define ALWAYS 288 -#define ANALYSE 289 -#define ANALYZE 290 -#define AND 291 -#define ANTI 292 -#define ANY 293 -#define ARRAY 294 -#define AS 295 -#define ASC_P 296 -#define ASOF 297 -#define ASSERTION 298 -#define ASSIGNMENT 299 -#define ASYMMETRIC 300 -#define AT 301 -#define ATTACH 302 -#define ATTRIBUTE 303 -#define AUTHORIZATION 304 -#define BACKWARD 305 -#define BEFORE 306 -#define BEGIN_P 307 -#define BETWEEN 308 -#define BIGINT 309 -#define BINARY 310 -#define BIT 311 -#define BOOLEAN_P 312 -#define BOTH 313 -#define BY 314 -#define CACHE 315 -#define CALL_P 316 -#define CALLED 317 -#define CASCADE 318 -#define CASCADED 319 -#define CASE 320 -#define CAST 321 -#define CATALOG_P 322 -#define CENTURIES_P 323 -#define CENTURY_P 324 -#define CHAIN 325 -#define CHAR_P 326 -#define CHARACTER 327 -#define CHARACTERISTICS 328 -#define CHECK_P 329 -#define CHECKPOINT 330 -#define CLASS 331 -#define CLOSE 332 -#define CLUSTER 333 -#define COALESCE 334 -#define COLLATE 335 -#define COLLATION 336 -#define COLUMN 337 -#define COLUMNS 338 -#define COMMENT 339 -#define COMMENTS 340 -#define COMMIT 341 -#define COMMITTED 342 -#define COMPRESSION 343 -#define CONCURRENTLY 344 -#define CONFIGURATION 345 -#define CONFLICT 346 -#define CONNECTION 347 -#define CONSTRAINT 348 -#define CONSTRAINTS 349 -#define CONTENT_P 350 -#define CONTINUE_P 351 -#define CONVERSION_P 352 -#define COPY 353 -#define COST 354 -#define CREATE_P 355 -#define CROSS 356 -#define CSV 357 -#define CUBE 358 -#define CURRENT_P 359 -#define CURSOR 360 -#define CYCLE 361 -#define DATA_P 362 -#define DATABASE 363 -#define DAY_P 364 -#define DAYS_P 365 -#define DEALLOCATE 366 -#define DEC 367 -#define DECADE_P 368 -#define DECADES_P 369 -#define DECIMAL_P 370 -#define DECLARE 371 -#define DEFAULT 372 -#define DEFAULTS 373 -#define DEFERRABLE 374 -#define DEFERRED 375 -#define DEFINER 376 -#define DELETE_P 377 -#define DELIMITER 378 -#define DELIMITERS 379 -#define DEPENDS 380 -#define DESC_P 381 -#define DESCRIBE 382 -#define DETACH 383 -#define DICTIONARY 384 -#define DISABLE_P 385 -#define DISCARD 386 -#define DISTINCT 387 -#define DO 388 -#define DOCUMENT_P 389 -#define DOMAIN_P 390 -#define DOUBLE_P 391 -#define DROP 392 -#define EACH 393 -#define ELSE 394 -#define ENABLE_P 395 -#define ENCODING 396 -#define ENCRYPTED 397 -#define END_P 398 -#define ENUM_P 399 -#define ESCAPE 400 -#define EVENT 401 -#define EXCEPT 402 -#define EXCLUDE 403 -#define EXCLUDING 404 -#define EXCLUSIVE 405 -#define EXECUTE 406 -#define EXISTS 407 -#define EXPLAIN 408 -#define EXPORT_P 409 -#define EXPORT_STATE 410 -#define EXTENSION 411 -#define EXTERNAL 412 -#define EXTRACT 413 -#define FALSE_P 414 -#define FAMILY 415 -#define FETCH 416 -#define FILTER 417 -#define FIRST_P 418 -#define FLOAT_P 419 -#define FOLLOWING 420 -#define FOR 421 -#define FORCE 422 -#define FOREIGN 423 -#define FORWARD 424 -#define FREEZE 425 -#define FROM 426 -#define FULL 427 -#define FUNCTION 428 -#define FUNCTIONS 429 -#define GENERATED 430 -#define GLOB 431 -#define GLOBAL 432 -#define GRANT 433 -#define GRANTED 434 -#define GROUP_P 435 -#define GROUPING 436 -#define GROUPING_ID 437 -#define GROUPS 438 -#define HANDLER 439 -#define HAVING 440 -#define HEADER_P 441 -#define HOLD 442 -#define HOUR_P 443 -#define HOURS_P 444 -#define IDENTITY_P 445 -#define IF_P 446 -#define IGNORE_P 447 -#define ILIKE 448 -#define IMMEDIATE 449 -#define IMMUTABLE 450 -#define IMPLICIT_P 451 -#define IMPORT_P 452 -#define IN_P 453 -#define INCLUDE_P 454 -#define INCLUDING 455 -#define INCREMENT 456 -#define INDEX 457 -#define INDEXES 458 -#define INHERIT 459 -#define INHERITS 460 -#define INITIALLY 461 -#define INLINE_P 462 -#define INNER_P 463 -#define INOUT 464 -#define INPUT_P 465 -#define INSENSITIVE 466 -#define INSERT 467 -#define INSTALL 468 -#define INSTEAD 469 -#define INT_P 470 -#define INTEGER 471 -#define INTERSECT 472 -#define INTERVAL 473 -#define INTO 474 -#define INVOKER 475 -#define IS 476 -#define ISNULL 477 -#define ISOLATION 478 -#define JOIN 479 -#define JSON 480 -#define KEY 481 -#define LABEL 482 -#define LANGUAGE 483 -#define LARGE_P 484 -#define LAST_P 485 -#define LATERAL_P 486 -#define LEADING 487 -#define LEAKPROOF 488 -#define LEFT 489 -#define LEVEL 490 -#define LIKE 491 -#define LIMIT 492 -#define LISTEN 493 -#define LOAD 494 -#define LOCAL 495 -#define LOCATION 496 -#define LOCK_P 497 -#define LOCKED 498 -#define LOGGED 499 -#define MACRO 500 -#define MAP 501 -#define MAPPING 502 -#define MATCH 503 -#define MATERIALIZED 504 -#define MAXVALUE 505 -#define METHOD 506 -#define MICROSECOND_P 507 -#define MICROSECONDS_P 508 -#define MILLENNIA_P 509 -#define MILLENNIUM_P 510 -#define MILLISECOND_P 511 -#define MILLISECONDS_P 512 -#define MINUTE_P 513 -#define MINUTES_P 514 -#define MINVALUE 515 -#define MODE 516 -#define MONTH_P 517 -#define MONTHS_P 518 -#define MOVE 519 -#define NAME_P 520 -#define NAMES 521 -#define NATIONAL 522 -#define NATURAL 523 -#define NCHAR 524 -#define NEW 525 -#define NEXT 526 -#define NO 527 -#define NONE 528 -#define NOT 529 -#define NOTHING 530 -#define NOTIFY 531 -#define NOTNULL 532 -#define NOWAIT 533 -#define NULL_P 534 -#define NULLIF 535 -#define NULLS_P 536 -#define NUMERIC 537 -#define OBJECT_P 538 -#define OF 539 -#define OFF 540 -#define OFFSET 541 -#define OIDS 542 -#define OLD 543 -#define ON 544 -#define ONLY 545 -#define OPERATOR 546 -#define OPTION 547 -#define OPTIONS 548 -#define OR 549 -#define ORDER 550 -#define ORDINALITY 551 -#define OTHERS 552 -#define OUT_P 553 -#define OUTER_P 554 -#define OVER 555 -#define OVERLAPS 556 -#define OVERLAY 557 -#define OVERRIDING 558 -#define OWNED 559 -#define OWNER 560 -#define PARALLEL 561 -#define PARSER 562 -#define PARTIAL 563 -#define PARTITION 564 -#define PASSING 565 -#define PASSWORD 566 -#define PERCENT 567 -#define PIVOT 568 -#define PIVOT_LONGER 569 -#define PIVOT_WIDER 570 -#define PLACING 571 -#define PLANS 572 -#define POLICY 573 -#define POSITION 574 -#define POSITIONAL 575 -#define PRAGMA_P 576 -#define PRECEDING 577 -#define PRECISION 578 -#define PREPARE 579 -#define PREPARED 580 -#define PRESERVE 581 -#define PRIMARY 582 -#define PRIOR 583 -#define PRIVILEGES 584 -#define PROCEDURAL 585 -#define PROCEDURE 586 -#define PROGRAM 587 -#define PUBLICATION 588 -#define QUALIFY 589 -#define QUOTE 590 -#define RANGE 591 -#define READ_P 592 -#define REAL 593 -#define REASSIGN 594 -#define RECHECK 595 -#define RECURSIVE 596 -#define REF 597 -#define REFERENCES 598 -#define REFERENCING 599 -#define REFRESH 600 -#define REINDEX 601 -#define RELATIVE_P 602 -#define RELEASE 603 -#define RENAME 604 -#define REPEATABLE 605 -#define REPLACE 606 -#define REPLICA 607 -#define RESET 608 -#define RESPECT_P 609 -#define RESTART 610 -#define RESTRICT 611 -#define RETURNING 612 -#define RETURNS 613 -#define REVOKE 614 -#define RIGHT 615 -#define ROLE 616 -#define ROLLBACK 617 -#define ROLLUP 618 -#define ROW 619 -#define ROWS 620 -#define RULE 621 -#define SAMPLE 622 -#define SAVEPOINT 623 -#define SCHEMA 624 -#define SCHEMAS 625 -#define SCROLL 626 -#define SEARCH 627 -#define SECOND_P 628 -#define SECONDS_P 629 -#define SECURITY 630 -#define SELECT 631 -#define SEMI 632 -#define SEQUENCE 633 -#define SEQUENCES 634 -#define SERIALIZABLE 635 -#define SERVER 636 -#define SESSION 637 -#define SET 638 -#define SETOF 639 -#define SETS 640 -#define SHARE 641 -#define SHOW 642 -#define SIMILAR 643 -#define SIMPLE 644 -#define SKIP 645 -#define SMALLINT 646 -#define SNAPSHOT 647 -#define SOME 648 -#define SQL_P 649 -#define STABLE 650 -#define STANDALONE_P 651 -#define START 652 -#define STATEMENT 653 -#define STATISTICS 654 -#define STDIN 655 -#define STDOUT 656 -#define STORAGE 657 -#define STORED 658 -#define STRICT_P 659 -#define STRIP_P 660 -#define STRUCT 661 -#define SUBSCRIPTION 662 -#define SUBSTRING 663 -#define SUMMARIZE 664 -#define SYMMETRIC 665 -#define SYSID 666 -#define SYSTEM_P 667 -#define TABLE 668 -#define TABLES 669 -#define TABLESAMPLE 670 -#define TABLESPACE 671 -#define TEMP 672 -#define TEMPLATE 673 -#define TEMPORARY 674 -#define TEXT_P 675 -#define THEN 676 -#define TIES 677 -#define TIME 678 -#define TIMESTAMP 679 -#define TO 680 -#define TRAILING 681 -#define TRANSACTION 682 -#define TRANSFORM 683 -#define TREAT 684 -#define TRIGGER 685 -#define TRIM 686 -#define TRUE_P 687 -#define TRUNCATE 688 -#define TRUSTED 689 -#define TRY_CAST 690 -#define TYPE_P 691 -#define TYPES_P 692 -#define UNBOUNDED 693 -#define UNCOMMITTED 694 -#define UNENCRYPTED 695 -#define UNION 696 -#define UNIQUE 697 -#define UNKNOWN 698 -#define UNLISTEN 699 -#define UNLOGGED 700 -#define UNPIVOT 701 -#define UNTIL 702 -#define UPDATE 703 -#define USE_P 704 -#define USER 705 -#define USING 706 -#define VACUUM 707 -#define VALID 708 -#define VALIDATE 709 -#define VALIDATOR 710 -#define VALUE_P 711 -#define VALUES 712 -#define VARCHAR 713 -#define VARIADIC 714 -#define VARYING 715 -#define VERBOSE 716 -#define VERSION_P 717 -#define VIEW 718 -#define VIEWS 719 -#define VIRTUAL 720 -#define VOLATILE 721 -#define WEEK_P 722 -#define WEEKS_P 723 -#define WHEN 724 -#define WHERE 725 -#define WHITESPACE_P 726 -#define WINDOW 727 -#define WITH 728 -#define WITHIN 729 -#define WITHOUT 730 -#define WORK 731 -#define WRAPPER 732 -#define WRITE_P 733 -#define XML_P 734 -#define XMLATTRIBUTES 735 -#define XMLCONCAT 736 -#define XMLELEMENT 737 -#define XMLEXISTS 738 -#define XMLFOREST 739 -#define XMLNAMESPACES 740 -#define XMLPARSE 741 -#define XMLPI 742 -#define XMLROOT 743 -#define XMLSERIALIZE 744 -#define XMLTABLE 745 -#define YEAR_P 746 -#define YEARS_P 747 -#define YES_P 748 -#define ZONE 749 -#define NOT_LA 750 -#define NULLS_LA 751 -#define WITH_LA 752 -#define POSTFIXOP 753 -#define UMINUS 754 - - - +/* Value type. */ #if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED -typedef union YYSTYPE -#line 14 "third_party/libpg_query/grammar/grammar.y" +union YYSTYPE { +#line 14 "third_party/libpg_query/grammar/grammar.y" + core_YYSTYPE core_yystype; /* these fields must match core_YYSTYPE: */ int ival; @@ -1089,28 +609,33 @@ typedef union YYSTYPE PGSubLinkType subquerytype; PGViewCheckOption viewcheckoption; PGInsertColumnOrder bynameorposition; -} -/* Line 1529 of yacc.c. */ -#line 1095 "third_party/libpg_query/grammar/grammar_out.hpp" - YYSTYPE; -# define yystype YYSTYPE /* obsolescent; will be withdrawn */ -# define YYSTYPE_IS_DECLARED 1 -# define YYSTYPE_IS_TRIVIAL 1 -#endif +#line 614 "third_party/libpg_query/grammar/grammar_out.hpp" +}; +typedef union YYSTYPE YYSTYPE; +# define YYSTYPE_IS_TRIVIAL 1 +# define YYSTYPE_IS_DECLARED 1 +#endif +/* Location type. */ #if ! defined YYLTYPE && ! defined YYLTYPE_IS_DECLARED -typedef struct YYLTYPE +typedef struct YYLTYPE YYLTYPE; +struct YYLTYPE { int first_line; int first_column; int last_line; int last_column; -} YYLTYPE; -# define yyltype YYLTYPE /* obsolescent; will be withdrawn */ +}; # define YYLTYPE_IS_DECLARED 1 # define YYLTYPE_IS_TRIVIAL 1 #endif + + +int base_yyparse (core_yyscan_t yyscanner); + + +#endif /* !YY_BASE_YY_THIRD_PARTY_LIBPG_QUERY_GRAMMAR_GRAMMAR_OUT_HPP_INCLUDED */ 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 3111cedb7..e39ca2994 100644 --- a/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp +++ b/src/duckdb/third_party/libpg_query/include/parser/kwlist.hpp @@ -108,7 +108,7 @@ PG_KEYWORD("delimiter", DELIMITER, UNRESERVED_KEYWORD) PG_KEYWORD("delimiters", DELIMITERS, UNRESERVED_KEYWORD) PG_KEYWORD("depends", DEPENDS, UNRESERVED_KEYWORD) PG_KEYWORD("desc", DESC_P, RESERVED_KEYWORD) -PG_KEYWORD("describe", DESCRIBE, UNRESERVED_KEYWORD) +PG_KEYWORD("describe", DESCRIBE, RESERVED_KEYWORD) PG_KEYWORD("detach", DETACH, UNRESERVED_KEYWORD) PG_KEYWORD("dictionary", DICTIONARY, UNRESERVED_KEYWORD) PG_KEYWORD("disable", DISABLE_P, UNRESERVED_KEYWORD) @@ -294,6 +294,7 @@ PG_KEYWORD("partition", PARTITION, UNRESERVED_KEYWORD) PG_KEYWORD("passing", PASSING, UNRESERVED_KEYWORD) PG_KEYWORD("password", PASSWORD, UNRESERVED_KEYWORD) PG_KEYWORD("percent", PERCENT, UNRESERVED_KEYWORD) +PG_KEYWORD("persistent", PERSISTENT, UNRESERVED_KEYWORD) PG_KEYWORD("pivot", PIVOT, RESERVED_KEYWORD) PG_KEYWORD("pivot_longer", PIVOT_LONGER, RESERVED_KEYWORD) PG_KEYWORD("pivot_wider", PIVOT_WIDER, RESERVED_KEYWORD) @@ -352,10 +353,12 @@ PG_KEYWORD("sample", SAMPLE, UNRESERVED_KEYWORD) PG_KEYWORD("savepoint", SAVEPOINT, UNRESERVED_KEYWORD) PG_KEYWORD("schema", SCHEMA, UNRESERVED_KEYWORD) PG_KEYWORD("schemas", SCHEMAS, UNRESERVED_KEYWORD) +PG_KEYWORD("scope", SCOPE, UNRESERVED_KEYWORD) PG_KEYWORD("scroll", SCROLL, UNRESERVED_KEYWORD) PG_KEYWORD("search", SEARCH, UNRESERVED_KEYWORD) PG_KEYWORD("second", SECOND_P, UNRESERVED_KEYWORD) PG_KEYWORD("seconds", SECONDS_P, UNRESERVED_KEYWORD) +PG_KEYWORD("secret", SECRET, UNRESERVED_KEYWORD) PG_KEYWORD("security", SECURITY, UNRESERVED_KEYWORD) PG_KEYWORD("select", SELECT, RESERVED_KEYWORD) PG_KEYWORD("semi", SEMI, TYPE_FUNC_NAME_KEYWORD) @@ -368,7 +371,7 @@ PG_KEYWORD("set", SET, UNRESERVED_KEYWORD) PG_KEYWORD("setof", SETOF, COL_NAME_KEYWORD) PG_KEYWORD("sets", SETS, UNRESERVED_KEYWORD) PG_KEYWORD("share", SHARE, UNRESERVED_KEYWORD) -PG_KEYWORD("show", SHOW, UNRESERVED_KEYWORD) +PG_KEYWORD("show", SHOW, RESERVED_KEYWORD) PG_KEYWORD("similar", SIMILAR, TYPE_FUNC_NAME_KEYWORD) PG_KEYWORD("simple", SIMPLE, UNRESERVED_KEYWORD) PG_KEYWORD("skip", SKIP, UNRESERVED_KEYWORD) @@ -390,7 +393,7 @@ PG_KEYWORD("strip", STRIP_P, UNRESERVED_KEYWORD) PG_KEYWORD("struct", STRUCT, TYPE_FUNC_NAME_KEYWORD) PG_KEYWORD("subscription", SUBSCRIPTION, UNRESERVED_KEYWORD) PG_KEYWORD("substring", SUBSTRING, COL_NAME_KEYWORD) -PG_KEYWORD("summarize", SUMMARIZE, UNRESERVED_KEYWORD) +PG_KEYWORD("summarize", SUMMARIZE, RESERVED_KEYWORD) PG_KEYWORD("symmetric", SYMMETRIC, RESERVED_KEYWORD) PG_KEYWORD("sysid", SYSID, UNRESERVED_KEYWORD) PG_KEYWORD("system", SYSTEM_P, UNRESERVED_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 7b37589e5..a67207543 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 @@ -1,14 +1,14 @@ -/* A Bison parser, made by GNU Bison 2.3. */ +/* A Bison parser, made by GNU Bison 3.8.2. */ -/* Skeleton implementation for Bison's Yacc-like parsers in C +/* Bison implementation for Yacc-like parsers in C - Copyright (C) 1984, 1989, 1990, 2000, 2001, 2002, 2003, 2004, 2005, 2006 - Free Software Foundation, Inc. + Copyright (C) 1984, 1989-1990, 2000-2015, 2018-2021 Free Software Foundation, + Inc. - This program is free software; you can redistribute it and/or modify + This program is free software: you can redistribute it and/or modify it under the terms of the GNU General Public License as published by - the Free Software Foundation; either version 2, or (at your option) - any later version. + the Free Software Foundation, either version 3 of the License, or + (at your option) any later version. This program is distributed in the hope that it will be useful, but WITHOUT ANY WARRANTY; without even the implied warranty of @@ -16,9 +16,7 @@ GNU General Public License for more details. You should have received a copy of the GNU General Public License - along with this program; if not, write to the Free Software - Foundation, Inc., 51 Franklin Street, Fifth Floor, - Boston, MA 02110-1301, USA. */ + along with this program. If not, see . */ /* As a special exception, you may create a larger work that contains part or all of the Bison parser skeleton and distribute that work @@ -36,6 +34,10 @@ /* C LALR(1) parser skeleton written by Richard Stallman, by simplifying the original so-called "semantic" parser. */ +/* DO NOT RELY ON FEATURES THAT ARE NOT DOCUMENTED in the manual, + especially those whose name start with YY_ or yy_. They are + private implementation details that can be changed or removed. */ + /* All symbols defined below should begin with yy or YY, to avoid infringing on user name space. This should be done even for local variables, as they might otherwise be expanded by user macros. @@ -43,11 +45,11 @@ define necessary library symbols; they are noted "INFRINGES ON USER NAME SPACE" below. */ -/* Identify Bison output. */ -#define YYBISON 1 +/* Identify Bison output, and Bison version. */ +#define YYBISON 30802 -/* Bison version. */ -#define YYBISON_VERSION "2.3" +/* Bison version string. */ +#define YYBISON_VERSION "3.8.2" /* Skeleton name. */ #define YYSKELETON_NAME "yacc.c" @@ -55,1027 +57,21 @@ /* Pure parsers. */ #define YYPURE 1 -/* Using locations. */ -#define YYLSP_NEEDED 1 +/* Push parsers. */ +#define YYPUSH 0 + +/* Pull parsers. */ +#define YYPULL 1 + /* Substitute the variable and function names. */ -#define yyparse base_yyparse -#define yylex base_yylex -#define yyerror base_yyerror -#define yylval base_yylval -#define yychar base_yychar -#define yydebug base_yydebug -#define yynerrs base_yynerrs -#define yylloc base_yylloc - -/* Tokens. */ -#ifndef YYTOKENTYPE -# define YYTOKENTYPE - /* Put the tokens into the symbol table, so that GDB and other debuggers - know about them. */ - enum yytokentype { - IDENT = 258, - FCONST = 259, - SCONST = 260, - BCONST = 261, - XCONST = 262, - Op = 263, - ICONST = 264, - PARAM = 265, - TYPECAST = 266, - DOT_DOT = 267, - COLON_EQUALS = 268, - EQUALS_GREATER = 269, - INTEGER_DIVISION = 270, - POWER_OF = 271, - LAMBDA_ARROW = 272, - DOUBLE_ARROW = 273, - LESS_EQUALS = 274, - GREATER_EQUALS = 275, - NOT_EQUALS = 276, - ABORT_P = 277, - ABSOLUTE_P = 278, - ACCESS = 279, - ACTION = 280, - ADD_P = 281, - ADMIN = 282, - AFTER = 283, - AGGREGATE = 284, - ALL = 285, - ALSO = 286, - ALTER = 287, - ALWAYS = 288, - ANALYSE = 289, - ANALYZE = 290, - AND = 291, - ANTI = 292, - ANY = 293, - ARRAY = 294, - AS = 295, - ASC_P = 296, - ASOF = 297, - ASSERTION = 298, - ASSIGNMENT = 299, - ASYMMETRIC = 300, - AT = 301, - ATTACH = 302, - ATTRIBUTE = 303, - AUTHORIZATION = 304, - BACKWARD = 305, - BEFORE = 306, - BEGIN_P = 307, - BETWEEN = 308, - BIGINT = 309, - BINARY = 310, - BIT = 311, - BOOLEAN_P = 312, - BOTH = 313, - BY = 314, - CACHE = 315, - CALL_P = 316, - CALLED = 317, - CASCADE = 318, - CASCADED = 319, - CASE = 320, - CAST = 321, - CATALOG_P = 322, - CENTURIES_P = 323, - CENTURY_P = 324, - CHAIN = 325, - CHAR_P = 326, - CHARACTER = 327, - CHARACTERISTICS = 328, - CHECK_P = 329, - CHECKPOINT = 330, - CLASS = 331, - CLOSE = 332, - CLUSTER = 333, - COALESCE = 334, - COLLATE = 335, - COLLATION = 336, - COLUMN = 337, - COLUMNS = 338, - COMMENT = 339, - COMMENTS = 340, - COMMIT = 341, - COMMITTED = 342, - COMPRESSION = 343, - CONCURRENTLY = 344, - CONFIGURATION = 345, - CONFLICT = 346, - CONNECTION = 347, - CONSTRAINT = 348, - CONSTRAINTS = 349, - CONTENT_P = 350, - CONTINUE_P = 351, - CONVERSION_P = 352, - COPY = 353, - COST = 354, - CREATE_P = 355, - CROSS = 356, - CSV = 357, - CUBE = 358, - CURRENT_P = 359, - CURSOR = 360, - CYCLE = 361, - DATA_P = 362, - DATABASE = 363, - DAY_P = 364, - DAYS_P = 365, - DEALLOCATE = 366, - DEC = 367, - DECADE_P = 368, - DECADES_P = 369, - DECIMAL_P = 370, - DECLARE = 371, - DEFAULT = 372, - DEFAULTS = 373, - DEFERRABLE = 374, - DEFERRED = 375, - DEFINER = 376, - DELETE_P = 377, - DELIMITER = 378, - DELIMITERS = 379, - DEPENDS = 380, - DESC_P = 381, - DESCRIBE = 382, - DETACH = 383, - DICTIONARY = 384, - DISABLE_P = 385, - DISCARD = 386, - DISTINCT = 387, - DO = 388, - DOCUMENT_P = 389, - DOMAIN_P = 390, - DOUBLE_P = 391, - DROP = 392, - EACH = 393, - ELSE = 394, - ENABLE_P = 395, - ENCODING = 396, - ENCRYPTED = 397, - END_P = 398, - ENUM_P = 399, - ESCAPE = 400, - EVENT = 401, - EXCEPT = 402, - EXCLUDE = 403, - EXCLUDING = 404, - EXCLUSIVE = 405, - EXECUTE = 406, - EXISTS = 407, - EXPLAIN = 408, - EXPORT_P = 409, - EXPORT_STATE = 410, - EXTENSION = 411, - EXTERNAL = 412, - EXTRACT = 413, - FALSE_P = 414, - FAMILY = 415, - FETCH = 416, - FILTER = 417, - FIRST_P = 418, - FLOAT_P = 419, - FOLLOWING = 420, - FOR = 421, - FORCE = 422, - FOREIGN = 423, - FORWARD = 424, - FREEZE = 425, - FROM = 426, - FULL = 427, - FUNCTION = 428, - FUNCTIONS = 429, - GENERATED = 430, - GLOB = 431, - GLOBAL = 432, - GRANT = 433, - GRANTED = 434, - GROUP_P = 435, - GROUPING = 436, - GROUPING_ID = 437, - GROUPS = 438, - HANDLER = 439, - HAVING = 440, - HEADER_P = 441, - HOLD = 442, - HOUR_P = 443, - HOURS_P = 444, - IDENTITY_P = 445, - IF_P = 446, - IGNORE_P = 447, - ILIKE = 448, - IMMEDIATE = 449, - IMMUTABLE = 450, - IMPLICIT_P = 451, - IMPORT_P = 452, - IN_P = 453, - INCLUDE_P = 454, - INCLUDING = 455, - INCREMENT = 456, - INDEX = 457, - INDEXES = 458, - INHERIT = 459, - INHERITS = 460, - INITIALLY = 461, - INLINE_P = 462, - INNER_P = 463, - INOUT = 464, - INPUT_P = 465, - INSENSITIVE = 466, - INSERT = 467, - INSTALL = 468, - INSTEAD = 469, - INT_P = 470, - INTEGER = 471, - INTERSECT = 472, - INTERVAL = 473, - INTO = 474, - INVOKER = 475, - IS = 476, - ISNULL = 477, - ISOLATION = 478, - JOIN = 479, - JSON = 480, - KEY = 481, - LABEL = 482, - LANGUAGE = 483, - LARGE_P = 484, - LAST_P = 485, - LATERAL_P = 486, - LEADING = 487, - LEAKPROOF = 488, - LEFT = 489, - LEVEL = 490, - LIKE = 491, - LIMIT = 492, - LISTEN = 493, - LOAD = 494, - LOCAL = 495, - LOCATION = 496, - LOCK_P = 497, - LOCKED = 498, - LOGGED = 499, - MACRO = 500, - MAP = 501, - MAPPING = 502, - MATCH = 503, - MATERIALIZED = 504, - MAXVALUE = 505, - METHOD = 506, - MICROSECOND_P = 507, - MICROSECONDS_P = 508, - MILLENNIA_P = 509, - MILLENNIUM_P = 510, - MILLISECOND_P = 511, - MILLISECONDS_P = 512, - MINUTE_P = 513, - MINUTES_P = 514, - MINVALUE = 515, - MODE = 516, - MONTH_P = 517, - MONTHS_P = 518, - MOVE = 519, - NAME_P = 520, - NAMES = 521, - NATIONAL = 522, - NATURAL = 523, - NCHAR = 524, - NEW = 525, - NEXT = 526, - NO = 527, - NONE = 528, - NOT = 529, - NOTHING = 530, - NOTIFY = 531, - NOTNULL = 532, - NOWAIT = 533, - NULL_P = 534, - NULLIF = 535, - NULLS_P = 536, - NUMERIC = 537, - OBJECT_P = 538, - OF = 539, - OFF = 540, - OFFSET = 541, - OIDS = 542, - OLD = 543, - ON = 544, - ONLY = 545, - OPERATOR = 546, - OPTION = 547, - OPTIONS = 548, - OR = 549, - ORDER = 550, - ORDINALITY = 551, - OTHERS = 552, - OUT_P = 553, - OUTER_P = 554, - OVER = 555, - OVERLAPS = 556, - OVERLAY = 557, - OVERRIDING = 558, - OWNED = 559, - OWNER = 560, - PARALLEL = 561, - PARSER = 562, - PARTIAL = 563, - PARTITION = 564, - PASSING = 565, - PASSWORD = 566, - PERCENT = 567, - PIVOT = 568, - PIVOT_LONGER = 569, - PIVOT_WIDER = 570, - PLACING = 571, - PLANS = 572, - POLICY = 573, - POSITION = 574, - POSITIONAL = 575, - PRAGMA_P = 576, - PRECEDING = 577, - PRECISION = 578, - PREPARE = 579, - PREPARED = 580, - PRESERVE = 581, - PRIMARY = 582, - PRIOR = 583, - PRIVILEGES = 584, - PROCEDURAL = 585, - PROCEDURE = 586, - PROGRAM = 587, - PUBLICATION = 588, - QUALIFY = 589, - QUOTE = 590, - RANGE = 591, - READ_P = 592, - REAL = 593, - REASSIGN = 594, - RECHECK = 595, - RECURSIVE = 596, - REF = 597, - REFERENCES = 598, - REFERENCING = 599, - REFRESH = 600, - REINDEX = 601, - RELATIVE_P = 602, - RELEASE = 603, - RENAME = 604, - REPEATABLE = 605, - REPLACE = 606, - REPLICA = 607, - RESET = 608, - RESPECT_P = 609, - RESTART = 610, - RESTRICT = 611, - RETURNING = 612, - RETURNS = 613, - REVOKE = 614, - RIGHT = 615, - ROLE = 616, - ROLLBACK = 617, - ROLLUP = 618, - ROW = 619, - ROWS = 620, - RULE = 621, - SAMPLE = 622, - SAVEPOINT = 623, - SCHEMA = 624, - SCHEMAS = 625, - SCROLL = 626, - SEARCH = 627, - SECOND_P = 628, - SECONDS_P = 629, - SECURITY = 630, - SELECT = 631, - SEMI = 632, - SEQUENCE = 633, - SEQUENCES = 634, - SERIALIZABLE = 635, - SERVER = 636, - SESSION = 637, - SET = 638, - SETOF = 639, - SETS = 640, - SHARE = 641, - SHOW = 642, - SIMILAR = 643, - SIMPLE = 644, - SKIP = 645, - SMALLINT = 646, - SNAPSHOT = 647, - SOME = 648, - SQL_P = 649, - STABLE = 650, - STANDALONE_P = 651, - START = 652, - STATEMENT = 653, - STATISTICS = 654, - STDIN = 655, - STDOUT = 656, - STORAGE = 657, - STORED = 658, - STRICT_P = 659, - STRIP_P = 660, - STRUCT = 661, - SUBSCRIPTION = 662, - SUBSTRING = 663, - SUMMARIZE = 664, - SYMMETRIC = 665, - SYSID = 666, - SYSTEM_P = 667, - TABLE = 668, - TABLES = 669, - TABLESAMPLE = 670, - TABLESPACE = 671, - TEMP = 672, - TEMPLATE = 673, - TEMPORARY = 674, - TEXT_P = 675, - THEN = 676, - TIES = 677, - TIME = 678, - TIMESTAMP = 679, - TO = 680, - TRAILING = 681, - TRANSACTION = 682, - TRANSFORM = 683, - TREAT = 684, - TRIGGER = 685, - TRIM = 686, - TRUE_P = 687, - TRUNCATE = 688, - TRUSTED = 689, - TRY_CAST = 690, - TYPE_P = 691, - TYPES_P = 692, - UNBOUNDED = 693, - UNCOMMITTED = 694, - UNENCRYPTED = 695, - UNION = 696, - UNIQUE = 697, - UNKNOWN = 698, - UNLISTEN = 699, - UNLOGGED = 700, - UNPIVOT = 701, - UNTIL = 702, - UPDATE = 703, - USE_P = 704, - USER = 705, - USING = 706, - VACUUM = 707, - VALID = 708, - VALIDATE = 709, - VALIDATOR = 710, - VALUE_P = 711, - VALUES = 712, - VARCHAR = 713, - VARIADIC = 714, - VARYING = 715, - VERBOSE = 716, - VERSION_P = 717, - VIEW = 718, - VIEWS = 719, - VIRTUAL = 720, - VOLATILE = 721, - WEEK_P = 722, - WEEKS_P = 723, - WHEN = 724, - WHERE = 725, - WHITESPACE_P = 726, - WINDOW = 727, - WITH = 728, - WITHIN = 729, - WITHOUT = 730, - WORK = 731, - WRAPPER = 732, - WRITE_P = 733, - XML_P = 734, - XMLATTRIBUTES = 735, - XMLCONCAT = 736, - XMLELEMENT = 737, - XMLEXISTS = 738, - XMLFOREST = 739, - XMLNAMESPACES = 740, - XMLPARSE = 741, - XMLPI = 742, - XMLROOT = 743, - XMLSERIALIZE = 744, - XMLTABLE = 745, - YEAR_P = 746, - YEARS_P = 747, - YES_P = 748, - ZONE = 749, - NOT_LA = 750, - NULLS_LA = 751, - WITH_LA = 752, - POSTFIXOP = 753, - UMINUS = 754 - }; -#endif -/* Tokens. */ -#define IDENT 258 -#define FCONST 259 -#define SCONST 260 -#define BCONST 261 -#define XCONST 262 -#define Op 263 -#define ICONST 264 -#define PARAM 265 -#define TYPECAST 266 -#define DOT_DOT 267 -#define COLON_EQUALS 268 -#define EQUALS_GREATER 269 -#define INTEGER_DIVISION 270 -#define POWER_OF 271 -#define LAMBDA_ARROW 272 -#define DOUBLE_ARROW 273 -#define LESS_EQUALS 274 -#define GREATER_EQUALS 275 -#define NOT_EQUALS 276 -#define ABORT_P 277 -#define ABSOLUTE_P 278 -#define ACCESS 279 -#define ACTION 280 -#define ADD_P 281 -#define ADMIN 282 -#define AFTER 283 -#define AGGREGATE 284 -#define ALL 285 -#define ALSO 286 -#define ALTER 287 -#define ALWAYS 288 -#define ANALYSE 289 -#define ANALYZE 290 -#define AND 291 -#define ANTI 292 -#define ANY 293 -#define ARRAY 294 -#define AS 295 -#define ASC_P 296 -#define ASOF 297 -#define ASSERTION 298 -#define ASSIGNMENT 299 -#define ASYMMETRIC 300 -#define AT 301 -#define ATTACH 302 -#define ATTRIBUTE 303 -#define AUTHORIZATION 304 -#define BACKWARD 305 -#define BEFORE 306 -#define BEGIN_P 307 -#define BETWEEN 308 -#define BIGINT 309 -#define BINARY 310 -#define BIT 311 -#define BOOLEAN_P 312 -#define BOTH 313 -#define BY 314 -#define CACHE 315 -#define CALL_P 316 -#define CALLED 317 -#define CASCADE 318 -#define CASCADED 319 -#define CASE 320 -#define CAST 321 -#define CATALOG_P 322 -#define CENTURIES_P 323 -#define CENTURY_P 324 -#define CHAIN 325 -#define CHAR_P 326 -#define CHARACTER 327 -#define CHARACTERISTICS 328 -#define CHECK_P 329 -#define CHECKPOINT 330 -#define CLASS 331 -#define CLOSE 332 -#define CLUSTER 333 -#define COALESCE 334 -#define COLLATE 335 -#define COLLATION 336 -#define COLUMN 337 -#define COLUMNS 338 -#define COMMENT 339 -#define COMMENTS 340 -#define COMMIT 341 -#define COMMITTED 342 -#define COMPRESSION 343 -#define CONCURRENTLY 344 -#define CONFIGURATION 345 -#define CONFLICT 346 -#define CONNECTION 347 -#define CONSTRAINT 348 -#define CONSTRAINTS 349 -#define CONTENT_P 350 -#define CONTINUE_P 351 -#define CONVERSION_P 352 -#define COPY 353 -#define COST 354 -#define CREATE_P 355 -#define CROSS 356 -#define CSV 357 -#define CUBE 358 -#define CURRENT_P 359 -#define CURSOR 360 -#define CYCLE 361 -#define DATA_P 362 -#define DATABASE 363 -#define DAY_P 364 -#define DAYS_P 365 -#define DEALLOCATE 366 -#define DEC 367 -#define DECADE_P 368 -#define DECADES_P 369 -#define DECIMAL_P 370 -#define DECLARE 371 -#define DEFAULT 372 -#define DEFAULTS 373 -#define DEFERRABLE 374 -#define DEFERRED 375 -#define DEFINER 376 -#define DELETE_P 377 -#define DELIMITER 378 -#define DELIMITERS 379 -#define DEPENDS 380 -#define DESC_P 381 -#define DESCRIBE 382 -#define DETACH 383 -#define DICTIONARY 384 -#define DISABLE_P 385 -#define DISCARD 386 -#define DISTINCT 387 -#define DO 388 -#define DOCUMENT_P 389 -#define DOMAIN_P 390 -#define DOUBLE_P 391 -#define DROP 392 -#define EACH 393 -#define ELSE 394 -#define ENABLE_P 395 -#define ENCODING 396 -#define ENCRYPTED 397 -#define END_P 398 -#define ENUM_P 399 -#define ESCAPE 400 -#define EVENT 401 -#define EXCEPT 402 -#define EXCLUDE 403 -#define EXCLUDING 404 -#define EXCLUSIVE 405 -#define EXECUTE 406 -#define EXISTS 407 -#define EXPLAIN 408 -#define EXPORT_P 409 -#define EXPORT_STATE 410 -#define EXTENSION 411 -#define EXTERNAL 412 -#define EXTRACT 413 -#define FALSE_P 414 -#define FAMILY 415 -#define FETCH 416 -#define FILTER 417 -#define FIRST_P 418 -#define FLOAT_P 419 -#define FOLLOWING 420 -#define FOR 421 -#define FORCE 422 -#define FOREIGN 423 -#define FORWARD 424 -#define FREEZE 425 -#define FROM 426 -#define FULL 427 -#define FUNCTION 428 -#define FUNCTIONS 429 -#define GENERATED 430 -#define GLOB 431 -#define GLOBAL 432 -#define GRANT 433 -#define GRANTED 434 -#define GROUP_P 435 -#define GROUPING 436 -#define GROUPING_ID 437 -#define GROUPS 438 -#define HANDLER 439 -#define HAVING 440 -#define HEADER_P 441 -#define HOLD 442 -#define HOUR_P 443 -#define HOURS_P 444 -#define IDENTITY_P 445 -#define IF_P 446 -#define IGNORE_P 447 -#define ILIKE 448 -#define IMMEDIATE 449 -#define IMMUTABLE 450 -#define IMPLICIT_P 451 -#define IMPORT_P 452 -#define IN_P 453 -#define INCLUDE_P 454 -#define INCLUDING 455 -#define INCREMENT 456 -#define INDEX 457 -#define INDEXES 458 -#define INHERIT 459 -#define INHERITS 460 -#define INITIALLY 461 -#define INLINE_P 462 -#define INNER_P 463 -#define INOUT 464 -#define INPUT_P 465 -#define INSENSITIVE 466 -#define INSERT 467 -#define INSTALL 468 -#define INSTEAD 469 -#define INT_P 470 -#define INTEGER 471 -#define INTERSECT 472 -#define INTERVAL 473 -#define INTO 474 -#define INVOKER 475 -#define IS 476 -#define ISNULL 477 -#define ISOLATION 478 -#define JOIN 479 -#define JSON 480 -#define KEY 481 -#define LABEL 482 -#define LANGUAGE 483 -#define LARGE_P 484 -#define LAST_P 485 -#define LATERAL_P 486 -#define LEADING 487 -#define LEAKPROOF 488 -#define LEFT 489 -#define LEVEL 490 -#define LIKE 491 -#define LIMIT 492 -#define LISTEN 493 -#define LOAD 494 -#define LOCAL 495 -#define LOCATION 496 -#define LOCK_P 497 -#define LOCKED 498 -#define LOGGED 499 -#define MACRO 500 -#define MAP 501 -#define MAPPING 502 -#define MATCH 503 -#define MATERIALIZED 504 -#define MAXVALUE 505 -#define METHOD 506 -#define MICROSECOND_P 507 -#define MICROSECONDS_P 508 -#define MILLENNIA_P 509 -#define MILLENNIUM_P 510 -#define MILLISECOND_P 511 -#define MILLISECONDS_P 512 -#define MINUTE_P 513 -#define MINUTES_P 514 -#define MINVALUE 515 -#define MODE 516 -#define MONTH_P 517 -#define MONTHS_P 518 -#define MOVE 519 -#define NAME_P 520 -#define NAMES 521 -#define NATIONAL 522 -#define NATURAL 523 -#define NCHAR 524 -#define NEW 525 -#define NEXT 526 -#define NO 527 -#define NONE 528 -#define NOT 529 -#define NOTHING 530 -#define NOTIFY 531 -#define NOTNULL 532 -#define NOWAIT 533 -#define NULL_P 534 -#define NULLIF 535 -#define NULLS_P 536 -#define NUMERIC 537 -#define OBJECT_P 538 -#define OF 539 -#define OFF 540 -#define OFFSET 541 -#define OIDS 542 -#define OLD 543 -#define ON 544 -#define ONLY 545 -#define OPERATOR 546 -#define OPTION 547 -#define OPTIONS 548 -#define OR 549 -#define ORDER 550 -#define ORDINALITY 551 -#define OTHERS 552 -#define OUT_P 553 -#define OUTER_P 554 -#define OVER 555 -#define OVERLAPS 556 -#define OVERLAY 557 -#define OVERRIDING 558 -#define OWNED 559 -#define OWNER 560 -#define PARALLEL 561 -#define PARSER 562 -#define PARTIAL 563 -#define PARTITION 564 -#define PASSING 565 -#define PASSWORD 566 -#define PERCENT 567 -#define PIVOT 568 -#define PIVOT_LONGER 569 -#define PIVOT_WIDER 570 -#define PLACING 571 -#define PLANS 572 -#define POLICY 573 -#define POSITION 574 -#define POSITIONAL 575 -#define PRAGMA_P 576 -#define PRECEDING 577 -#define PRECISION 578 -#define PREPARE 579 -#define PREPARED 580 -#define PRESERVE 581 -#define PRIMARY 582 -#define PRIOR 583 -#define PRIVILEGES 584 -#define PROCEDURAL 585 -#define PROCEDURE 586 -#define PROGRAM 587 -#define PUBLICATION 588 -#define QUALIFY 589 -#define QUOTE 590 -#define RANGE 591 -#define READ_P 592 -#define REAL 593 -#define REASSIGN 594 -#define RECHECK 595 -#define RECURSIVE 596 -#define REF 597 -#define REFERENCES 598 -#define REFERENCING 599 -#define REFRESH 600 -#define REINDEX 601 -#define RELATIVE_P 602 -#define RELEASE 603 -#define RENAME 604 -#define REPEATABLE 605 -#define REPLACE 606 -#define REPLICA 607 -#define RESET 608 -#define RESPECT_P 609 -#define RESTART 610 -#define RESTRICT 611 -#define RETURNING 612 -#define RETURNS 613 -#define REVOKE 614 -#define RIGHT 615 -#define ROLE 616 -#define ROLLBACK 617 -#define ROLLUP 618 -#define ROW 619 -#define ROWS 620 -#define RULE 621 -#define SAMPLE 622 -#define SAVEPOINT 623 -#define SCHEMA 624 -#define SCHEMAS 625 -#define SCROLL 626 -#define SEARCH 627 -#define SECOND_P 628 -#define SECONDS_P 629 -#define SECURITY 630 -#define SELECT 631 -#define SEMI 632 -#define SEQUENCE 633 -#define SEQUENCES 634 -#define SERIALIZABLE 635 -#define SERVER 636 -#define SESSION 637 -#define SET 638 -#define SETOF 639 -#define SETS 640 -#define SHARE 641 -#define SHOW 642 -#define SIMILAR 643 -#define SIMPLE 644 -#define SKIP 645 -#define SMALLINT 646 -#define SNAPSHOT 647 -#define SOME 648 -#define SQL_P 649 -#define STABLE 650 -#define STANDALONE_P 651 -#define START 652 -#define STATEMENT 653 -#define STATISTICS 654 -#define STDIN 655 -#define STDOUT 656 -#define STORAGE 657 -#define STORED 658 -#define STRICT_P 659 -#define STRIP_P 660 -#define STRUCT 661 -#define SUBSCRIPTION 662 -#define SUBSTRING 663 -#define SUMMARIZE 664 -#define SYMMETRIC 665 -#define SYSID 666 -#define SYSTEM_P 667 -#define TABLE 668 -#define TABLES 669 -#define TABLESAMPLE 670 -#define TABLESPACE 671 -#define TEMP 672 -#define TEMPLATE 673 -#define TEMPORARY 674 -#define TEXT_P 675 -#define THEN 676 -#define TIES 677 -#define TIME 678 -#define TIMESTAMP 679 -#define TO 680 -#define TRAILING 681 -#define TRANSACTION 682 -#define TRANSFORM 683 -#define TREAT 684 -#define TRIGGER 685 -#define TRIM 686 -#define TRUE_P 687 -#define TRUNCATE 688 -#define TRUSTED 689 -#define TRY_CAST 690 -#define TYPE_P 691 -#define TYPES_P 692 -#define UNBOUNDED 693 -#define UNCOMMITTED 694 -#define UNENCRYPTED 695 -#define UNION 696 -#define UNIQUE 697 -#define UNKNOWN 698 -#define UNLISTEN 699 -#define UNLOGGED 700 -#define UNPIVOT 701 -#define UNTIL 702 -#define UPDATE 703 -#define USE_P 704 -#define USER 705 -#define USING 706 -#define VACUUM 707 -#define VALID 708 -#define VALIDATE 709 -#define VALIDATOR 710 -#define VALUE_P 711 -#define VALUES 712 -#define VARCHAR 713 -#define VARIADIC 714 -#define VARYING 715 -#define VERBOSE 716 -#define VERSION_P 717 -#define VIEW 718 -#define VIEWS 719 -#define VIRTUAL 720 -#define VOLATILE 721 -#define WEEK_P 722 -#define WEEKS_P 723 -#define WHEN 724 -#define WHERE 725 -#define WHITESPACE_P 726 -#define WINDOW 727 -#define WITH 728 -#define WITHIN 729 -#define WITHOUT 730 -#define WORK 731 -#define WRAPPER 732 -#define WRITE_P 733 -#define XML_P 734 -#define XMLATTRIBUTES 735 -#define XMLCONCAT 736 -#define XMLELEMENT 737 -#define XMLEXISTS 738 -#define XMLFOREST 739 -#define XMLNAMESPACES 740 -#define XMLPARSE 741 -#define XMLPI 742 -#define XMLROOT 743 -#define XMLSERIALIZE 744 -#define XMLTABLE 745 -#define YEAR_P 746 -#define YEARS_P 747 -#define YES_P 748 -#define ZONE 749 -#define NOT_LA 750 -#define NULLS_LA 751 -#define WITH_LA 752 -#define POSTFIXOP 753 -#define UMINUS 754 - - - - -/* Copy the first part of user declarations. */ +#define yyparse base_yyparse +#define yylex base_yylex +#define yyerror base_yyerror +#define yydebug base_yydebug +#define yynerrs base_yynerrs + +/* First part of user prologue. */ #line 1 "third_party/libpg_query/grammar/grammar.y.tmp" #line 1 "third_party/libpg_query/grammar/grammar.hpp" @@ -1241,132 +237,1108 @@ static PGNode *makeRecursiveViewSelect(char *relname, PGList *aliases, PGNode *q static PGNode *makeLimitPercent(PGNode *limit_percent); +#line 241 "third_party/libpg_query/grammar/grammar_out.cpp" -/* Enabling traces. */ -#ifndef YYDEBUG -# define YYDEBUG 0 -#endif - -/* Enabling verbose error messages. */ -#ifdef YYERROR_VERBOSE -# undef YYERROR_VERBOSE -# define YYERROR_VERBOSE 1 -#else -# define YYERROR_VERBOSE 0 -#endif - -/* Enabling the token table. */ -#ifndef YYTOKEN_TABLE -# define YYTOKEN_TABLE 0 -#endif - -#if ! defined YYSTYPE && ! defined YYSTYPE_IS_DECLARED -typedef union YYSTYPE -#line 14 "third_party/libpg_query/grammar/grammar.y" -{ - core_YYSTYPE core_yystype; - /* these fields must match core_YYSTYPE: */ - int ival; - char *str; - const char *keyword; - const char *conststr; - - char chr; - bool boolean; - PGJoinType jtype; - PGDropBehavior dbehavior; - PGOnCommitAction oncommit; - PGOnCreateConflict oncreateconflict; - PGList *list; - PGNode *node; - PGValue *value; - PGObjectType objtype; - PGTypeName *typnam; - PGObjectWithArgs *objwithargs; - PGDefElem *defelt; - PGSortBy *sortby; - PGWindowDef *windef; - PGJoinExpr *jexpr; - PGIndexElem *ielem; - PGAlias *alias; - PGRangeVar *range; - PGIntoClause *into; - PGCTEMaterialize ctematerialize; - PGWithClause *with; - PGInferClause *infer; - PGOnConflictClause *onconflict; - PGOnConflictActionAlias onconflictshorthand; - PGAIndices *aind; - PGResTarget *target; - PGInsertStmt *istmt; - PGVariableSetStmt *vsetstmt; - PGOverridingKind override; - PGSortByDir sortorder; - PGSortByNulls nullorder; - PGConstrType constr; - PGLockClauseStrength lockstrength; - PGLockWaitPolicy lockwaitpolicy; - PGSubLinkType subquerytype; - PGViewCheckOption viewcheckoption; - PGInsertColumnOrder bynameorposition; -} -/* Line 193 of yacc.c. */ -#line 1315 "third_party/libpg_query/grammar/grammar_out.cpp" - YYSTYPE; -# define yystype YYSTYPE /* obsolescent; will be withdrawn */ -# define YYSTYPE_IS_DECLARED 1 -# define YYSTYPE_IS_TRIVIAL 1 -#endif +# ifndef YY_CAST +# ifdef __cplusplus +# define YY_CAST(Type, Val) static_cast (Val) +# define YY_REINTERPRET_CAST(Type, Val) reinterpret_cast (Val) +# else +# define YY_CAST(Type, Val) ((Type) (Val)) +# define YY_REINTERPRET_CAST(Type, Val) ((Type) (Val)) +# endif +# endif +# ifndef YY_NULLPTR +# if defined __cplusplus +# if 201103L <= __cplusplus +# define YY_NULLPTR nullptr +# else +# define YY_NULLPTR 0 +# endif +# else +# define YY_NULLPTR ((void*)0) +# endif +# endif -#if ! defined YYLTYPE && ! defined YYLTYPE_IS_DECLARED -typedef struct YYLTYPE +#include "include/parser/gram.hpp" +/* Symbol kind. */ +enum yysymbol_kind_t { - int first_line; - int first_column; - int last_line; - int last_column; -} YYLTYPE; -# define yyltype YYLTYPE /* obsolescent; will be withdrawn */ -# define YYLTYPE_IS_DECLARED 1 -# define YYLTYPE_IS_TRIVIAL 1 -#endif - + YYSYMBOL_YYEMPTY = -2, + YYSYMBOL_YYEOF = 0, /* "end of file" */ + YYSYMBOL_YYerror = 1, /* error */ + YYSYMBOL_YYUNDEF = 2, /* "invalid token" */ + YYSYMBOL_IDENT = 3, /* IDENT */ + YYSYMBOL_FCONST = 4, /* FCONST */ + YYSYMBOL_SCONST = 5, /* SCONST */ + YYSYMBOL_BCONST = 6, /* BCONST */ + YYSYMBOL_XCONST = 7, /* XCONST */ + YYSYMBOL_Op = 8, /* Op */ + YYSYMBOL_ICONST = 9, /* ICONST */ + YYSYMBOL_PARAM = 10, /* PARAM */ + YYSYMBOL_TYPECAST = 11, /* TYPECAST */ + YYSYMBOL_DOT_DOT = 12, /* DOT_DOT */ + YYSYMBOL_COLON_EQUALS = 13, /* COLON_EQUALS */ + YYSYMBOL_EQUALS_GREATER = 14, /* EQUALS_GREATER */ + YYSYMBOL_INTEGER_DIVISION = 15, /* INTEGER_DIVISION */ + YYSYMBOL_POWER_OF = 16, /* POWER_OF */ + YYSYMBOL_LAMBDA_ARROW = 17, /* LAMBDA_ARROW */ + YYSYMBOL_DOUBLE_ARROW = 18, /* DOUBLE_ARROW */ + YYSYMBOL_LESS_EQUALS = 19, /* LESS_EQUALS */ + YYSYMBOL_GREATER_EQUALS = 20, /* GREATER_EQUALS */ + YYSYMBOL_NOT_EQUALS = 21, /* NOT_EQUALS */ + YYSYMBOL_ABORT_P = 22, /* ABORT_P */ + YYSYMBOL_ABSOLUTE_P = 23, /* ABSOLUTE_P */ + YYSYMBOL_ACCESS = 24, /* ACCESS */ + YYSYMBOL_ACTION = 25, /* ACTION */ + YYSYMBOL_ADD_P = 26, /* ADD_P */ + YYSYMBOL_ADMIN = 27, /* ADMIN */ + YYSYMBOL_AFTER = 28, /* AFTER */ + YYSYMBOL_AGGREGATE = 29, /* AGGREGATE */ + YYSYMBOL_ALL = 30, /* ALL */ + YYSYMBOL_ALSO = 31, /* ALSO */ + YYSYMBOL_ALTER = 32, /* ALTER */ + YYSYMBOL_ALWAYS = 33, /* ALWAYS */ + YYSYMBOL_ANALYSE = 34, /* ANALYSE */ + YYSYMBOL_ANALYZE = 35, /* ANALYZE */ + YYSYMBOL_AND = 36, /* AND */ + YYSYMBOL_ANTI = 37, /* ANTI */ + YYSYMBOL_ANY = 38, /* ANY */ + YYSYMBOL_ARRAY = 39, /* ARRAY */ + YYSYMBOL_AS = 40, /* AS */ + YYSYMBOL_ASC_P = 41, /* ASC_P */ + YYSYMBOL_ASOF = 42, /* ASOF */ + YYSYMBOL_ASSERTION = 43, /* ASSERTION */ + YYSYMBOL_ASSIGNMENT = 44, /* ASSIGNMENT */ + YYSYMBOL_ASYMMETRIC = 45, /* ASYMMETRIC */ + YYSYMBOL_AT = 46, /* AT */ + YYSYMBOL_ATTACH = 47, /* ATTACH */ + YYSYMBOL_ATTRIBUTE = 48, /* ATTRIBUTE */ + YYSYMBOL_AUTHORIZATION = 49, /* AUTHORIZATION */ + YYSYMBOL_BACKWARD = 50, /* BACKWARD */ + YYSYMBOL_BEFORE = 51, /* BEFORE */ + YYSYMBOL_BEGIN_P = 52, /* BEGIN_P */ + YYSYMBOL_BETWEEN = 53, /* BETWEEN */ + YYSYMBOL_BIGINT = 54, /* BIGINT */ + YYSYMBOL_BINARY = 55, /* BINARY */ + YYSYMBOL_BIT = 56, /* BIT */ + YYSYMBOL_BOOLEAN_P = 57, /* BOOLEAN_P */ + YYSYMBOL_BOTH = 58, /* BOTH */ + YYSYMBOL_BY = 59, /* BY */ + YYSYMBOL_CACHE = 60, /* CACHE */ + YYSYMBOL_CALL_P = 61, /* CALL_P */ + YYSYMBOL_CALLED = 62, /* CALLED */ + YYSYMBOL_CASCADE = 63, /* CASCADE */ + YYSYMBOL_CASCADED = 64, /* CASCADED */ + YYSYMBOL_CASE = 65, /* CASE */ + YYSYMBOL_CAST = 66, /* CAST */ + YYSYMBOL_CATALOG_P = 67, /* CATALOG_P */ + YYSYMBOL_CENTURIES_P = 68, /* CENTURIES_P */ + YYSYMBOL_CENTURY_P = 69, /* CENTURY_P */ + YYSYMBOL_CHAIN = 70, /* CHAIN */ + YYSYMBOL_CHAR_P = 71, /* CHAR_P */ + YYSYMBOL_CHARACTER = 72, /* CHARACTER */ + YYSYMBOL_CHARACTERISTICS = 73, /* CHARACTERISTICS */ + YYSYMBOL_CHECK_P = 74, /* CHECK_P */ + YYSYMBOL_CHECKPOINT = 75, /* CHECKPOINT */ + YYSYMBOL_CLASS = 76, /* CLASS */ + YYSYMBOL_CLOSE = 77, /* CLOSE */ + YYSYMBOL_CLUSTER = 78, /* CLUSTER */ + YYSYMBOL_COALESCE = 79, /* COALESCE */ + YYSYMBOL_COLLATE = 80, /* COLLATE */ + YYSYMBOL_COLLATION = 81, /* COLLATION */ + YYSYMBOL_COLUMN = 82, /* COLUMN */ + YYSYMBOL_COLUMNS = 83, /* COLUMNS */ + YYSYMBOL_COMMENT = 84, /* COMMENT */ + YYSYMBOL_COMMENTS = 85, /* COMMENTS */ + YYSYMBOL_COMMIT = 86, /* COMMIT */ + YYSYMBOL_COMMITTED = 87, /* COMMITTED */ + YYSYMBOL_COMPRESSION = 88, /* COMPRESSION */ + YYSYMBOL_CONCURRENTLY = 89, /* CONCURRENTLY */ + YYSYMBOL_CONFIGURATION = 90, /* CONFIGURATION */ + YYSYMBOL_CONFLICT = 91, /* CONFLICT */ + YYSYMBOL_CONNECTION = 92, /* CONNECTION */ + YYSYMBOL_CONSTRAINT = 93, /* CONSTRAINT */ + YYSYMBOL_CONSTRAINTS = 94, /* CONSTRAINTS */ + YYSYMBOL_CONTENT_P = 95, /* CONTENT_P */ + YYSYMBOL_CONTINUE_P = 96, /* CONTINUE_P */ + YYSYMBOL_CONVERSION_P = 97, /* CONVERSION_P */ + YYSYMBOL_COPY = 98, /* COPY */ + YYSYMBOL_COST = 99, /* COST */ + YYSYMBOL_CREATE_P = 100, /* CREATE_P */ + YYSYMBOL_CROSS = 101, /* CROSS */ + YYSYMBOL_CSV = 102, /* CSV */ + YYSYMBOL_CUBE = 103, /* CUBE */ + YYSYMBOL_CURRENT_P = 104, /* CURRENT_P */ + YYSYMBOL_CURSOR = 105, /* CURSOR */ + YYSYMBOL_CYCLE = 106, /* CYCLE */ + YYSYMBOL_DATA_P = 107, /* DATA_P */ + YYSYMBOL_DATABASE = 108, /* DATABASE */ + YYSYMBOL_DAY_P = 109, /* DAY_P */ + YYSYMBOL_DAYS_P = 110, /* DAYS_P */ + YYSYMBOL_DEALLOCATE = 111, /* DEALLOCATE */ + YYSYMBOL_DEC = 112, /* DEC */ + YYSYMBOL_DECADE_P = 113, /* DECADE_P */ + YYSYMBOL_DECADES_P = 114, /* DECADES_P */ + YYSYMBOL_DECIMAL_P = 115, /* DECIMAL_P */ + YYSYMBOL_DECLARE = 116, /* DECLARE */ + YYSYMBOL_DEFAULT = 117, /* DEFAULT */ + YYSYMBOL_DEFAULTS = 118, /* DEFAULTS */ + YYSYMBOL_DEFERRABLE = 119, /* DEFERRABLE */ + YYSYMBOL_DEFERRED = 120, /* DEFERRED */ + YYSYMBOL_DEFINER = 121, /* DEFINER */ + YYSYMBOL_DELETE_P = 122, /* DELETE_P */ + YYSYMBOL_DELIMITER = 123, /* DELIMITER */ + YYSYMBOL_DELIMITERS = 124, /* DELIMITERS */ + YYSYMBOL_DEPENDS = 125, /* DEPENDS */ + YYSYMBOL_DESC_P = 126, /* DESC_P */ + YYSYMBOL_DESCRIBE = 127, /* DESCRIBE */ + YYSYMBOL_DETACH = 128, /* DETACH */ + YYSYMBOL_DICTIONARY = 129, /* DICTIONARY */ + YYSYMBOL_DISABLE_P = 130, /* DISABLE_P */ + YYSYMBOL_DISCARD = 131, /* DISCARD */ + YYSYMBOL_DISTINCT = 132, /* DISTINCT */ + YYSYMBOL_DO = 133, /* DO */ + YYSYMBOL_DOCUMENT_P = 134, /* DOCUMENT_P */ + YYSYMBOL_DOMAIN_P = 135, /* DOMAIN_P */ + YYSYMBOL_DOUBLE_P = 136, /* DOUBLE_P */ + YYSYMBOL_DROP = 137, /* DROP */ + YYSYMBOL_EACH = 138, /* EACH */ + YYSYMBOL_ELSE = 139, /* ELSE */ + YYSYMBOL_ENABLE_P = 140, /* ENABLE_P */ + YYSYMBOL_ENCODING = 141, /* ENCODING */ + YYSYMBOL_ENCRYPTED = 142, /* ENCRYPTED */ + YYSYMBOL_END_P = 143, /* END_P */ + YYSYMBOL_ENUM_P = 144, /* ENUM_P */ + YYSYMBOL_ESCAPE = 145, /* ESCAPE */ + YYSYMBOL_EVENT = 146, /* EVENT */ + YYSYMBOL_EXCEPT = 147, /* EXCEPT */ + YYSYMBOL_EXCLUDE = 148, /* EXCLUDE */ + YYSYMBOL_EXCLUDING = 149, /* EXCLUDING */ + YYSYMBOL_EXCLUSIVE = 150, /* EXCLUSIVE */ + YYSYMBOL_EXECUTE = 151, /* EXECUTE */ + YYSYMBOL_EXISTS = 152, /* EXISTS */ + YYSYMBOL_EXPLAIN = 153, /* EXPLAIN */ + YYSYMBOL_EXPORT_P = 154, /* EXPORT_P */ + YYSYMBOL_EXPORT_STATE = 155, /* EXPORT_STATE */ + YYSYMBOL_EXTENSION = 156, /* EXTENSION */ + YYSYMBOL_EXTERNAL = 157, /* EXTERNAL */ + YYSYMBOL_EXTRACT = 158, /* EXTRACT */ + YYSYMBOL_FALSE_P = 159, /* FALSE_P */ + YYSYMBOL_FAMILY = 160, /* FAMILY */ + YYSYMBOL_FETCH = 161, /* FETCH */ + YYSYMBOL_FILTER = 162, /* FILTER */ + YYSYMBOL_FIRST_P = 163, /* FIRST_P */ + YYSYMBOL_FLOAT_P = 164, /* FLOAT_P */ + YYSYMBOL_FOLLOWING = 165, /* FOLLOWING */ + YYSYMBOL_FOR = 166, /* FOR */ + YYSYMBOL_FORCE = 167, /* FORCE */ + YYSYMBOL_FOREIGN = 168, /* FOREIGN */ + YYSYMBOL_FORWARD = 169, /* FORWARD */ + YYSYMBOL_FREEZE = 170, /* FREEZE */ + YYSYMBOL_FROM = 171, /* FROM */ + YYSYMBOL_FULL = 172, /* FULL */ + YYSYMBOL_FUNCTION = 173, /* FUNCTION */ + YYSYMBOL_FUNCTIONS = 174, /* FUNCTIONS */ + YYSYMBOL_GENERATED = 175, /* GENERATED */ + YYSYMBOL_GLOB = 176, /* GLOB */ + YYSYMBOL_GLOBAL = 177, /* GLOBAL */ + YYSYMBOL_GRANT = 178, /* GRANT */ + YYSYMBOL_GRANTED = 179, /* GRANTED */ + YYSYMBOL_GROUP_P = 180, /* GROUP_P */ + YYSYMBOL_GROUPING = 181, /* GROUPING */ + YYSYMBOL_GROUPING_ID = 182, /* GROUPING_ID */ + YYSYMBOL_GROUPS = 183, /* GROUPS */ + YYSYMBOL_HANDLER = 184, /* HANDLER */ + YYSYMBOL_HAVING = 185, /* HAVING */ + YYSYMBOL_HEADER_P = 186, /* HEADER_P */ + YYSYMBOL_HOLD = 187, /* HOLD */ + YYSYMBOL_HOUR_P = 188, /* HOUR_P */ + YYSYMBOL_HOURS_P = 189, /* HOURS_P */ + YYSYMBOL_IDENTITY_P = 190, /* IDENTITY_P */ + YYSYMBOL_IF_P = 191, /* IF_P */ + YYSYMBOL_IGNORE_P = 192, /* IGNORE_P */ + YYSYMBOL_ILIKE = 193, /* ILIKE */ + YYSYMBOL_IMMEDIATE = 194, /* IMMEDIATE */ + YYSYMBOL_IMMUTABLE = 195, /* IMMUTABLE */ + YYSYMBOL_IMPLICIT_P = 196, /* IMPLICIT_P */ + YYSYMBOL_IMPORT_P = 197, /* IMPORT_P */ + YYSYMBOL_IN_P = 198, /* IN_P */ + YYSYMBOL_INCLUDE_P = 199, /* INCLUDE_P */ + YYSYMBOL_INCLUDING = 200, /* INCLUDING */ + YYSYMBOL_INCREMENT = 201, /* INCREMENT */ + YYSYMBOL_INDEX = 202, /* INDEX */ + YYSYMBOL_INDEXES = 203, /* INDEXES */ + YYSYMBOL_INHERIT = 204, /* INHERIT */ + YYSYMBOL_INHERITS = 205, /* INHERITS */ + YYSYMBOL_INITIALLY = 206, /* INITIALLY */ + YYSYMBOL_INLINE_P = 207, /* INLINE_P */ + YYSYMBOL_INNER_P = 208, /* INNER_P */ + YYSYMBOL_INOUT = 209, /* INOUT */ + YYSYMBOL_INPUT_P = 210, /* INPUT_P */ + YYSYMBOL_INSENSITIVE = 211, /* INSENSITIVE */ + YYSYMBOL_INSERT = 212, /* INSERT */ + YYSYMBOL_INSTALL = 213, /* INSTALL */ + YYSYMBOL_INSTEAD = 214, /* INSTEAD */ + YYSYMBOL_INT_P = 215, /* INT_P */ + YYSYMBOL_INTEGER = 216, /* INTEGER */ + YYSYMBOL_INTERSECT = 217, /* INTERSECT */ + YYSYMBOL_INTERVAL = 218, /* INTERVAL */ + YYSYMBOL_INTO = 219, /* INTO */ + YYSYMBOL_INVOKER = 220, /* INVOKER */ + YYSYMBOL_IS = 221, /* IS */ + YYSYMBOL_ISNULL = 222, /* ISNULL */ + YYSYMBOL_ISOLATION = 223, /* ISOLATION */ + YYSYMBOL_JOIN = 224, /* JOIN */ + YYSYMBOL_JSON = 225, /* JSON */ + YYSYMBOL_KEY = 226, /* KEY */ + YYSYMBOL_LABEL = 227, /* LABEL */ + YYSYMBOL_LANGUAGE = 228, /* LANGUAGE */ + YYSYMBOL_LARGE_P = 229, /* LARGE_P */ + YYSYMBOL_LAST_P = 230, /* LAST_P */ + YYSYMBOL_LATERAL_P = 231, /* LATERAL_P */ + YYSYMBOL_LEADING = 232, /* LEADING */ + YYSYMBOL_LEAKPROOF = 233, /* LEAKPROOF */ + YYSYMBOL_LEFT = 234, /* LEFT */ + YYSYMBOL_LEVEL = 235, /* LEVEL */ + YYSYMBOL_LIKE = 236, /* LIKE */ + YYSYMBOL_LIMIT = 237, /* LIMIT */ + YYSYMBOL_LISTEN = 238, /* LISTEN */ + YYSYMBOL_LOAD = 239, /* LOAD */ + YYSYMBOL_LOCAL = 240, /* LOCAL */ + YYSYMBOL_LOCATION = 241, /* LOCATION */ + YYSYMBOL_LOCK_P = 242, /* LOCK_P */ + YYSYMBOL_LOCKED = 243, /* LOCKED */ + YYSYMBOL_LOGGED = 244, /* LOGGED */ + YYSYMBOL_MACRO = 245, /* MACRO */ + YYSYMBOL_MAP = 246, /* MAP */ + YYSYMBOL_MAPPING = 247, /* MAPPING */ + YYSYMBOL_MATCH = 248, /* MATCH */ + YYSYMBOL_MATERIALIZED = 249, /* MATERIALIZED */ + YYSYMBOL_MAXVALUE = 250, /* MAXVALUE */ + YYSYMBOL_METHOD = 251, /* METHOD */ + YYSYMBOL_MICROSECOND_P = 252, /* MICROSECOND_P */ + YYSYMBOL_MICROSECONDS_P = 253, /* MICROSECONDS_P */ + YYSYMBOL_MILLENNIA_P = 254, /* MILLENNIA_P */ + YYSYMBOL_MILLENNIUM_P = 255, /* MILLENNIUM_P */ + YYSYMBOL_MILLISECOND_P = 256, /* MILLISECOND_P */ + YYSYMBOL_MILLISECONDS_P = 257, /* MILLISECONDS_P */ + YYSYMBOL_MINUTE_P = 258, /* MINUTE_P */ + YYSYMBOL_MINUTES_P = 259, /* MINUTES_P */ + YYSYMBOL_MINVALUE = 260, /* MINVALUE */ + YYSYMBOL_MODE = 261, /* MODE */ + YYSYMBOL_MONTH_P = 262, /* MONTH_P */ + YYSYMBOL_MONTHS_P = 263, /* MONTHS_P */ + YYSYMBOL_MOVE = 264, /* MOVE */ + YYSYMBOL_NAME_P = 265, /* NAME_P */ + YYSYMBOL_NAMES = 266, /* NAMES */ + YYSYMBOL_NATIONAL = 267, /* NATIONAL */ + YYSYMBOL_NATURAL = 268, /* NATURAL */ + YYSYMBOL_NCHAR = 269, /* NCHAR */ + YYSYMBOL_NEW = 270, /* NEW */ + YYSYMBOL_NEXT = 271, /* NEXT */ + YYSYMBOL_NO = 272, /* NO */ + YYSYMBOL_NONE = 273, /* NONE */ + YYSYMBOL_NOT = 274, /* NOT */ + YYSYMBOL_NOTHING = 275, /* NOTHING */ + YYSYMBOL_NOTIFY = 276, /* NOTIFY */ + YYSYMBOL_NOTNULL = 277, /* NOTNULL */ + YYSYMBOL_NOWAIT = 278, /* NOWAIT */ + YYSYMBOL_NULL_P = 279, /* NULL_P */ + YYSYMBOL_NULLIF = 280, /* NULLIF */ + YYSYMBOL_NULLS_P = 281, /* NULLS_P */ + YYSYMBOL_NUMERIC = 282, /* NUMERIC */ + YYSYMBOL_OBJECT_P = 283, /* OBJECT_P */ + YYSYMBOL_OF = 284, /* OF */ + YYSYMBOL_OFF = 285, /* OFF */ + YYSYMBOL_OFFSET = 286, /* OFFSET */ + YYSYMBOL_OIDS = 287, /* OIDS */ + YYSYMBOL_OLD = 288, /* OLD */ + YYSYMBOL_ON = 289, /* ON */ + YYSYMBOL_ONLY = 290, /* ONLY */ + YYSYMBOL_OPERATOR = 291, /* OPERATOR */ + YYSYMBOL_OPTION = 292, /* OPTION */ + YYSYMBOL_OPTIONS = 293, /* OPTIONS */ + YYSYMBOL_OR = 294, /* OR */ + YYSYMBOL_ORDER = 295, /* ORDER */ + YYSYMBOL_ORDINALITY = 296, /* ORDINALITY */ + YYSYMBOL_OTHERS = 297, /* OTHERS */ + YYSYMBOL_OUT_P = 298, /* OUT_P */ + YYSYMBOL_OUTER_P = 299, /* OUTER_P */ + YYSYMBOL_OVER = 300, /* OVER */ + YYSYMBOL_OVERLAPS = 301, /* OVERLAPS */ + YYSYMBOL_OVERLAY = 302, /* OVERLAY */ + YYSYMBOL_OVERRIDING = 303, /* OVERRIDING */ + YYSYMBOL_OWNED = 304, /* OWNED */ + YYSYMBOL_OWNER = 305, /* OWNER */ + YYSYMBOL_PARALLEL = 306, /* PARALLEL */ + YYSYMBOL_PARSER = 307, /* PARSER */ + YYSYMBOL_PARTIAL = 308, /* PARTIAL */ + YYSYMBOL_PARTITION = 309, /* PARTITION */ + YYSYMBOL_PASSING = 310, /* PASSING */ + YYSYMBOL_PASSWORD = 311, /* PASSWORD */ + YYSYMBOL_PERCENT = 312, /* PERCENT */ + YYSYMBOL_PERSISTENT = 313, /* PERSISTENT */ + YYSYMBOL_PIVOT = 314, /* PIVOT */ + YYSYMBOL_PIVOT_LONGER = 315, /* PIVOT_LONGER */ + YYSYMBOL_PIVOT_WIDER = 316, /* PIVOT_WIDER */ + YYSYMBOL_PLACING = 317, /* PLACING */ + YYSYMBOL_PLANS = 318, /* PLANS */ + YYSYMBOL_POLICY = 319, /* POLICY */ + YYSYMBOL_POSITION = 320, /* POSITION */ + YYSYMBOL_POSITIONAL = 321, /* POSITIONAL */ + YYSYMBOL_PRAGMA_P = 322, /* PRAGMA_P */ + YYSYMBOL_PRECEDING = 323, /* PRECEDING */ + YYSYMBOL_PRECISION = 324, /* PRECISION */ + YYSYMBOL_PREPARE = 325, /* PREPARE */ + YYSYMBOL_PREPARED = 326, /* PREPARED */ + YYSYMBOL_PRESERVE = 327, /* PRESERVE */ + YYSYMBOL_PRIMARY = 328, /* PRIMARY */ + YYSYMBOL_PRIOR = 329, /* PRIOR */ + YYSYMBOL_PRIVILEGES = 330, /* PRIVILEGES */ + YYSYMBOL_PROCEDURAL = 331, /* PROCEDURAL */ + YYSYMBOL_PROCEDURE = 332, /* PROCEDURE */ + YYSYMBOL_PROGRAM = 333, /* PROGRAM */ + YYSYMBOL_PUBLICATION = 334, /* PUBLICATION */ + YYSYMBOL_QUALIFY = 335, /* QUALIFY */ + YYSYMBOL_QUOTE = 336, /* QUOTE */ + YYSYMBOL_RANGE = 337, /* RANGE */ + YYSYMBOL_READ_P = 338, /* READ_P */ + YYSYMBOL_REAL = 339, /* REAL */ + YYSYMBOL_REASSIGN = 340, /* REASSIGN */ + YYSYMBOL_RECHECK = 341, /* RECHECK */ + YYSYMBOL_RECURSIVE = 342, /* RECURSIVE */ + YYSYMBOL_REF = 343, /* REF */ + YYSYMBOL_REFERENCES = 344, /* REFERENCES */ + YYSYMBOL_REFERENCING = 345, /* REFERENCING */ + YYSYMBOL_REFRESH = 346, /* REFRESH */ + YYSYMBOL_REINDEX = 347, /* REINDEX */ + YYSYMBOL_RELATIVE_P = 348, /* RELATIVE_P */ + YYSYMBOL_RELEASE = 349, /* RELEASE */ + YYSYMBOL_RENAME = 350, /* RENAME */ + YYSYMBOL_REPEATABLE = 351, /* REPEATABLE */ + YYSYMBOL_REPLACE = 352, /* REPLACE */ + YYSYMBOL_REPLICA = 353, /* REPLICA */ + YYSYMBOL_RESET = 354, /* RESET */ + YYSYMBOL_RESPECT_P = 355, /* RESPECT_P */ + YYSYMBOL_RESTART = 356, /* RESTART */ + YYSYMBOL_RESTRICT = 357, /* RESTRICT */ + YYSYMBOL_RETURNING = 358, /* RETURNING */ + YYSYMBOL_RETURNS = 359, /* RETURNS */ + YYSYMBOL_REVOKE = 360, /* REVOKE */ + YYSYMBOL_RIGHT = 361, /* RIGHT */ + YYSYMBOL_ROLE = 362, /* ROLE */ + YYSYMBOL_ROLLBACK = 363, /* ROLLBACK */ + YYSYMBOL_ROLLUP = 364, /* ROLLUP */ + YYSYMBOL_ROW = 365, /* ROW */ + YYSYMBOL_ROWS = 366, /* ROWS */ + YYSYMBOL_RULE = 367, /* RULE */ + YYSYMBOL_SAMPLE = 368, /* SAMPLE */ + YYSYMBOL_SAVEPOINT = 369, /* SAVEPOINT */ + YYSYMBOL_SCHEMA = 370, /* SCHEMA */ + YYSYMBOL_SCHEMAS = 371, /* SCHEMAS */ + YYSYMBOL_SCOPE = 372, /* SCOPE */ + YYSYMBOL_SCROLL = 373, /* SCROLL */ + YYSYMBOL_SEARCH = 374, /* SEARCH */ + YYSYMBOL_SECOND_P = 375, /* SECOND_P */ + YYSYMBOL_SECONDS_P = 376, /* SECONDS_P */ + YYSYMBOL_SECRET = 377, /* SECRET */ + YYSYMBOL_SECURITY = 378, /* SECURITY */ + YYSYMBOL_SELECT = 379, /* SELECT */ + YYSYMBOL_SEMI = 380, /* SEMI */ + YYSYMBOL_SEQUENCE = 381, /* SEQUENCE */ + YYSYMBOL_SEQUENCES = 382, /* SEQUENCES */ + YYSYMBOL_SERIALIZABLE = 383, /* SERIALIZABLE */ + YYSYMBOL_SERVER = 384, /* SERVER */ + YYSYMBOL_SESSION = 385, /* SESSION */ + YYSYMBOL_SET = 386, /* SET */ + YYSYMBOL_SETOF = 387, /* SETOF */ + YYSYMBOL_SETS = 388, /* SETS */ + YYSYMBOL_SHARE = 389, /* SHARE */ + YYSYMBOL_SHOW = 390, /* SHOW */ + YYSYMBOL_SIMILAR = 391, /* SIMILAR */ + YYSYMBOL_SIMPLE = 392, /* SIMPLE */ + YYSYMBOL_SKIP = 393, /* SKIP */ + YYSYMBOL_SMALLINT = 394, /* SMALLINT */ + YYSYMBOL_SNAPSHOT = 395, /* SNAPSHOT */ + YYSYMBOL_SOME = 396, /* SOME */ + YYSYMBOL_SQL_P = 397, /* SQL_P */ + YYSYMBOL_STABLE = 398, /* STABLE */ + YYSYMBOL_STANDALONE_P = 399, /* STANDALONE_P */ + YYSYMBOL_START = 400, /* START */ + YYSYMBOL_STATEMENT = 401, /* STATEMENT */ + YYSYMBOL_STATISTICS = 402, /* STATISTICS */ + YYSYMBOL_STDIN = 403, /* STDIN */ + YYSYMBOL_STDOUT = 404, /* STDOUT */ + YYSYMBOL_STORAGE = 405, /* STORAGE */ + YYSYMBOL_STORED = 406, /* STORED */ + YYSYMBOL_STRICT_P = 407, /* STRICT_P */ + YYSYMBOL_STRIP_P = 408, /* STRIP_P */ + YYSYMBOL_STRUCT = 409, /* STRUCT */ + YYSYMBOL_SUBSCRIPTION = 410, /* SUBSCRIPTION */ + YYSYMBOL_SUBSTRING = 411, /* SUBSTRING */ + YYSYMBOL_SUMMARIZE = 412, /* SUMMARIZE */ + YYSYMBOL_SYMMETRIC = 413, /* SYMMETRIC */ + YYSYMBOL_SYSID = 414, /* SYSID */ + YYSYMBOL_SYSTEM_P = 415, /* SYSTEM_P */ + YYSYMBOL_TABLE = 416, /* TABLE */ + YYSYMBOL_TABLES = 417, /* TABLES */ + YYSYMBOL_TABLESAMPLE = 418, /* TABLESAMPLE */ + YYSYMBOL_TABLESPACE = 419, /* TABLESPACE */ + YYSYMBOL_TEMP = 420, /* TEMP */ + YYSYMBOL_TEMPLATE = 421, /* TEMPLATE */ + YYSYMBOL_TEMPORARY = 422, /* TEMPORARY */ + YYSYMBOL_TEXT_P = 423, /* TEXT_P */ + YYSYMBOL_THEN = 424, /* THEN */ + YYSYMBOL_TIES = 425, /* TIES */ + YYSYMBOL_TIME = 426, /* TIME */ + YYSYMBOL_TIMESTAMP = 427, /* TIMESTAMP */ + YYSYMBOL_TO = 428, /* TO */ + YYSYMBOL_TRAILING = 429, /* TRAILING */ + YYSYMBOL_TRANSACTION = 430, /* TRANSACTION */ + YYSYMBOL_TRANSFORM = 431, /* TRANSFORM */ + YYSYMBOL_TREAT = 432, /* TREAT */ + YYSYMBOL_TRIGGER = 433, /* TRIGGER */ + YYSYMBOL_TRIM = 434, /* TRIM */ + YYSYMBOL_TRUE_P = 435, /* TRUE_P */ + YYSYMBOL_TRUNCATE = 436, /* TRUNCATE */ + YYSYMBOL_TRUSTED = 437, /* TRUSTED */ + YYSYMBOL_TRY_CAST = 438, /* TRY_CAST */ + YYSYMBOL_TYPE_P = 439, /* TYPE_P */ + YYSYMBOL_TYPES_P = 440, /* TYPES_P */ + YYSYMBOL_UNBOUNDED = 441, /* UNBOUNDED */ + YYSYMBOL_UNCOMMITTED = 442, /* UNCOMMITTED */ + YYSYMBOL_UNENCRYPTED = 443, /* UNENCRYPTED */ + YYSYMBOL_UNION = 444, /* UNION */ + YYSYMBOL_UNIQUE = 445, /* UNIQUE */ + YYSYMBOL_UNKNOWN = 446, /* UNKNOWN */ + YYSYMBOL_UNLISTEN = 447, /* UNLISTEN */ + YYSYMBOL_UNLOGGED = 448, /* UNLOGGED */ + YYSYMBOL_UNPIVOT = 449, /* UNPIVOT */ + YYSYMBOL_UNTIL = 450, /* UNTIL */ + YYSYMBOL_UPDATE = 451, /* UPDATE */ + YYSYMBOL_USE_P = 452, /* USE_P */ + YYSYMBOL_USER = 453, /* USER */ + YYSYMBOL_USING = 454, /* USING */ + YYSYMBOL_VACUUM = 455, /* VACUUM */ + YYSYMBOL_VALID = 456, /* VALID */ + YYSYMBOL_VALIDATE = 457, /* VALIDATE */ + YYSYMBOL_VALIDATOR = 458, /* VALIDATOR */ + YYSYMBOL_VALUE_P = 459, /* VALUE_P */ + YYSYMBOL_VALUES = 460, /* VALUES */ + YYSYMBOL_VARCHAR = 461, /* VARCHAR */ + YYSYMBOL_VARIADIC = 462, /* VARIADIC */ + YYSYMBOL_VARYING = 463, /* VARYING */ + YYSYMBOL_VERBOSE = 464, /* VERBOSE */ + YYSYMBOL_VERSION_P = 465, /* VERSION_P */ + YYSYMBOL_VIEW = 466, /* VIEW */ + YYSYMBOL_VIEWS = 467, /* VIEWS */ + YYSYMBOL_VIRTUAL = 468, /* VIRTUAL */ + YYSYMBOL_VOLATILE = 469, /* VOLATILE */ + YYSYMBOL_WEEK_P = 470, /* WEEK_P */ + YYSYMBOL_WEEKS_P = 471, /* WEEKS_P */ + YYSYMBOL_WHEN = 472, /* WHEN */ + YYSYMBOL_WHERE = 473, /* WHERE */ + YYSYMBOL_WHITESPACE_P = 474, /* WHITESPACE_P */ + YYSYMBOL_WINDOW = 475, /* WINDOW */ + YYSYMBOL_WITH = 476, /* WITH */ + YYSYMBOL_WITHIN = 477, /* WITHIN */ + YYSYMBOL_WITHOUT = 478, /* WITHOUT */ + YYSYMBOL_WORK = 479, /* WORK */ + YYSYMBOL_WRAPPER = 480, /* WRAPPER */ + YYSYMBOL_WRITE_P = 481, /* WRITE_P */ + YYSYMBOL_XML_P = 482, /* XML_P */ + YYSYMBOL_XMLATTRIBUTES = 483, /* XMLATTRIBUTES */ + YYSYMBOL_XMLCONCAT = 484, /* XMLCONCAT */ + YYSYMBOL_XMLELEMENT = 485, /* XMLELEMENT */ + YYSYMBOL_XMLEXISTS = 486, /* XMLEXISTS */ + YYSYMBOL_XMLFOREST = 487, /* XMLFOREST */ + YYSYMBOL_XMLNAMESPACES = 488, /* XMLNAMESPACES */ + YYSYMBOL_XMLPARSE = 489, /* XMLPARSE */ + YYSYMBOL_XMLPI = 490, /* XMLPI */ + YYSYMBOL_XMLROOT = 491, /* XMLROOT */ + YYSYMBOL_XMLSERIALIZE = 492, /* XMLSERIALIZE */ + YYSYMBOL_XMLTABLE = 493, /* XMLTABLE */ + YYSYMBOL_YEAR_P = 494, /* YEAR_P */ + YYSYMBOL_YEARS_P = 495, /* YEARS_P */ + YYSYMBOL_YES_P = 496, /* YES_P */ + YYSYMBOL_ZONE = 497, /* ZONE */ + YYSYMBOL_NOT_LA = 498, /* NOT_LA */ + YYSYMBOL_NULLS_LA = 499, /* NULLS_LA */ + YYSYMBOL_WITH_LA = 500, /* WITH_LA */ + YYSYMBOL_501_ = 501, /* '<' */ + YYSYMBOL_502_ = 502, /* '>' */ + YYSYMBOL_503_ = 503, /* '=' */ + YYSYMBOL_POSTFIXOP = 504, /* POSTFIXOP */ + YYSYMBOL_505_ = 505, /* '+' */ + YYSYMBOL_506_ = 506, /* '-' */ + YYSYMBOL_507_ = 507, /* '*' */ + YYSYMBOL_508_ = 508, /* '/' */ + YYSYMBOL_509_ = 509, /* '%' */ + YYSYMBOL_510_ = 510, /* '^' */ + YYSYMBOL_UMINUS = 511, /* UMINUS */ + YYSYMBOL_512_ = 512, /* '[' */ + YYSYMBOL_513_ = 513, /* ']' */ + YYSYMBOL_514_ = 514, /* '(' */ + YYSYMBOL_515_ = 515, /* ')' */ + YYSYMBOL_516_ = 516, /* '.' */ + YYSYMBOL_517_ = 517, /* ';' */ + YYSYMBOL_518_ = 518, /* ',' */ + YYSYMBOL_519_ = 519, /* '#' */ + YYSYMBOL_520_ = 520, /* '$' */ + YYSYMBOL_521_ = 521, /* '?' */ + YYSYMBOL_522_ = 522, /* '{' */ + YYSYMBOL_523_ = 523, /* '}' */ + YYSYMBOL_524_ = 524, /* ':' */ + YYSYMBOL_YYACCEPT = 525, /* $accept */ + YYSYMBOL_stmtblock = 526, /* stmtblock */ + YYSYMBOL_stmtmulti = 527, /* stmtmulti */ + YYSYMBOL_stmt = 528, /* stmt */ + YYSYMBOL_AlterObjectSchemaStmt = 529, /* AlterObjectSchemaStmt */ + YYSYMBOL_AlterSeqStmt = 530, /* AlterSeqStmt */ + YYSYMBOL_SeqOptList = 531, /* SeqOptList */ + YYSYMBOL_opt_with = 532, /* opt_with */ + YYSYMBOL_NumericOnly = 533, /* NumericOnly */ + YYSYMBOL_SeqOptElem = 534, /* SeqOptElem */ + YYSYMBOL_opt_by = 535, /* opt_by */ + YYSYMBOL_SignedIconst = 536, /* SignedIconst */ + YYSYMBOL_AlterTableStmt = 537, /* AlterTableStmt */ + YYSYMBOL_alter_identity_column_option_list = 538, /* alter_identity_column_option_list */ + YYSYMBOL_alter_column_default = 539, /* alter_column_default */ + YYSYMBOL_alter_identity_column_option = 540, /* alter_identity_column_option */ + YYSYMBOL_alter_generic_option_list = 541, /* alter_generic_option_list */ + YYSYMBOL_alter_table_cmd = 542, /* alter_table_cmd */ + YYSYMBOL_alter_using = 543, /* alter_using */ + YYSYMBOL_alter_generic_option_elem = 544, /* alter_generic_option_elem */ + YYSYMBOL_alter_table_cmds = 545, /* alter_table_cmds */ + YYSYMBOL_alter_generic_options = 546, /* alter_generic_options */ + YYSYMBOL_opt_set_data = 547, /* opt_set_data */ + YYSYMBOL_AnalyzeStmt = 548, /* AnalyzeStmt */ + YYSYMBOL_AttachStmt = 549, /* AttachStmt */ + YYSYMBOL_DetachStmt = 550, /* DetachStmt */ + YYSYMBOL_opt_database = 551, /* opt_database */ + YYSYMBOL_opt_database_alias = 552, /* opt_database_alias */ + YYSYMBOL_CallStmt = 553, /* CallStmt */ + YYSYMBOL_CheckPointStmt = 554, /* CheckPointStmt */ + YYSYMBOL_opt_col_id = 555, /* opt_col_id */ + YYSYMBOL_CommentOnStmt = 556, /* CommentOnStmt */ + YYSYMBOL_comment_value = 557, /* comment_value */ + YYSYMBOL_comment_on_type_any_name = 558, /* comment_on_type_any_name */ + YYSYMBOL_qualified_name = 559, /* qualified_name */ + YYSYMBOL_ColId = 560, /* ColId */ + YYSYMBOL_ColIdOrString = 561, /* ColIdOrString */ + YYSYMBOL_Sconst = 562, /* Sconst */ + YYSYMBOL_indirection = 563, /* indirection */ + YYSYMBOL_indirection_el = 564, /* indirection_el */ + YYSYMBOL_attr_name = 565, /* attr_name */ + YYSYMBOL_ColLabel = 566, /* ColLabel */ + YYSYMBOL_CopyStmt = 567, /* CopyStmt */ + YYSYMBOL_copy_database_flag = 568, /* copy_database_flag */ + YYSYMBOL_copy_from = 569, /* copy_from */ + YYSYMBOL_copy_delimiter = 570, /* copy_delimiter */ + YYSYMBOL_copy_generic_opt_arg_list = 571, /* copy_generic_opt_arg_list */ + YYSYMBOL_opt_using = 572, /* opt_using */ + YYSYMBOL_opt_as = 573, /* opt_as */ + YYSYMBOL_opt_program = 574, /* opt_program */ + YYSYMBOL_copy_options = 575, /* copy_options */ + YYSYMBOL_copy_generic_opt_arg = 576, /* copy_generic_opt_arg */ + YYSYMBOL_copy_generic_opt_elem = 577, /* copy_generic_opt_elem */ + YYSYMBOL_opt_oids = 578, /* opt_oids */ + YYSYMBOL_copy_opt_list = 579, /* copy_opt_list */ + YYSYMBOL_opt_binary = 580, /* opt_binary */ + YYSYMBOL_copy_opt_item = 581, /* copy_opt_item */ + YYSYMBOL_copy_generic_opt_arg_list_item = 582, /* copy_generic_opt_arg_list_item */ + YYSYMBOL_copy_file_name = 583, /* copy_file_name */ + YYSYMBOL_copy_generic_opt_list = 584, /* copy_generic_opt_list */ + YYSYMBOL_CreateStmt = 585, /* CreateStmt */ + YYSYMBOL_ConstraintAttributeSpec = 586, /* ConstraintAttributeSpec */ + YYSYMBOL_def_arg = 587, /* def_arg */ + YYSYMBOL_OptParenthesizedSeqOptList = 588, /* OptParenthesizedSeqOptList */ + YYSYMBOL_generic_option_arg = 589, /* generic_option_arg */ + YYSYMBOL_key_action = 590, /* key_action */ + YYSYMBOL_ColConstraint = 591, /* ColConstraint */ + YYSYMBOL_ColConstraintElem = 592, /* ColConstraintElem */ + YYSYMBOL_GeneratedColumnType = 593, /* GeneratedColumnType */ + YYSYMBOL_opt_GeneratedColumnType = 594, /* opt_GeneratedColumnType */ + YYSYMBOL_GeneratedConstraintElem = 595, /* GeneratedConstraintElem */ + YYSYMBOL_generic_option_elem = 596, /* generic_option_elem */ + YYSYMBOL_key_update = 597, /* key_update */ + YYSYMBOL_key_actions = 598, /* key_actions */ + YYSYMBOL_OnCommitOption = 599, /* OnCommitOption */ + YYSYMBOL_reloptions = 600, /* reloptions */ + YYSYMBOL_opt_no_inherit = 601, /* opt_no_inherit */ + YYSYMBOL_TableConstraint = 602, /* TableConstraint */ + YYSYMBOL_TableLikeOption = 603, /* TableLikeOption */ + YYSYMBOL_reloption_list = 604, /* reloption_list */ + YYSYMBOL_ExistingIndex = 605, /* ExistingIndex */ + YYSYMBOL_ConstraintAttr = 606, /* ConstraintAttr */ + YYSYMBOL_OptWith = 607, /* OptWith */ + YYSYMBOL_definition = 608, /* definition */ + YYSYMBOL_TableLikeOptionList = 609, /* TableLikeOptionList */ + YYSYMBOL_generic_option_name = 610, /* generic_option_name */ + YYSYMBOL_ConstraintAttributeElem = 611, /* ConstraintAttributeElem */ + YYSYMBOL_columnDef = 612, /* columnDef */ + YYSYMBOL_def_list = 613, /* def_list */ + YYSYMBOL_index_name = 614, /* index_name */ + YYSYMBOL_TableElement = 615, /* TableElement */ + YYSYMBOL_def_elem = 616, /* def_elem */ + YYSYMBOL_opt_definition = 617, /* opt_definition */ + YYSYMBOL_OptTableElementList = 618, /* OptTableElementList */ + YYSYMBOL_columnElem = 619, /* columnElem */ + YYSYMBOL_opt_column_list = 620, /* opt_column_list */ + YYSYMBOL_ColQualList = 621, /* ColQualList */ + YYSYMBOL_key_delete = 622, /* key_delete */ + YYSYMBOL_reloption_elem = 623, /* reloption_elem */ + YYSYMBOL_columnList = 624, /* columnList */ + YYSYMBOL_columnList_opt_comma = 625, /* columnList_opt_comma */ + YYSYMBOL_func_type = 626, /* func_type */ + YYSYMBOL_ConstraintElem = 627, /* ConstraintElem */ + YYSYMBOL_TableElementList = 628, /* TableElementList */ + YYSYMBOL_key_match = 629, /* key_match */ + YYSYMBOL_TableLikeClause = 630, /* TableLikeClause */ + YYSYMBOL_OptTemp = 631, /* OptTemp */ + YYSYMBOL_generated_when = 632, /* generated_when */ + YYSYMBOL_CreateAsStmt = 633, /* CreateAsStmt */ + YYSYMBOL_opt_with_data = 634, /* opt_with_data */ + YYSYMBOL_create_as_target = 635, /* create_as_target */ + YYSYMBOL_unreserved_keyword = 636, /* unreserved_keyword */ + YYSYMBOL_col_name_keyword = 637, /* col_name_keyword */ + YYSYMBOL_func_name_keyword = 638, /* func_name_keyword */ + YYSYMBOL_type_name_keyword = 639, /* type_name_keyword */ + YYSYMBOL_other_keyword = 640, /* other_keyword */ + YYSYMBOL_type_func_name_keyword = 641, /* type_func_name_keyword */ + YYSYMBOL_reserved_keyword = 642, /* reserved_keyword */ + YYSYMBOL_CreateFunctionStmt = 643, /* CreateFunctionStmt */ + YYSYMBOL_macro_alias = 644, /* macro_alias */ + YYSYMBOL_param_list = 645, /* param_list */ + YYSYMBOL_CreateSchemaStmt = 646, /* CreateSchemaStmt */ + YYSYMBOL_OptSchemaEltList = 647, /* OptSchemaEltList */ + YYSYMBOL_schema_stmt = 648, /* schema_stmt */ + YYSYMBOL_CreateSecretStmt = 649, /* CreateSecretStmt */ + YYSYMBOL_opt_secret_name = 650, /* opt_secret_name */ + YYSYMBOL_opt_persist = 651, /* opt_persist */ + YYSYMBOL_opt_storage_specifier = 652, /* opt_storage_specifier */ + YYSYMBOL_CreateSeqStmt = 653, /* CreateSeqStmt */ + YYSYMBOL_OptSeqOptList = 654, /* OptSeqOptList */ + YYSYMBOL_CreateTypeStmt = 655, /* CreateTypeStmt */ + YYSYMBOL_opt_enum_val_list = 656, /* opt_enum_val_list */ + YYSYMBOL_enum_val_list = 657, /* enum_val_list */ + YYSYMBOL_DeallocateStmt = 658, /* DeallocateStmt */ + YYSYMBOL_DeleteStmt = 659, /* DeleteStmt */ + YYSYMBOL_relation_expr_opt_alias = 660, /* relation_expr_opt_alias */ + YYSYMBOL_where_or_current_clause = 661, /* where_or_current_clause */ + YYSYMBOL_using_clause = 662, /* using_clause */ + YYSYMBOL_DropStmt = 663, /* DropStmt */ + YYSYMBOL_drop_type_any_name = 664, /* drop_type_any_name */ + YYSYMBOL_drop_type_name = 665, /* drop_type_name */ + YYSYMBOL_any_name_list = 666, /* any_name_list */ + YYSYMBOL_opt_drop_behavior = 667, /* opt_drop_behavior */ + YYSYMBOL_drop_type_name_on_any_name = 668, /* drop_type_name_on_any_name */ + YYSYMBOL_DropSecretStmt = 669, /* DropSecretStmt */ + YYSYMBOL_opt_storage_drop_specifier = 670, /* opt_storage_drop_specifier */ + YYSYMBOL_ExecuteStmt = 671, /* ExecuteStmt */ + YYSYMBOL_execute_param_expr = 672, /* execute_param_expr */ + YYSYMBOL_execute_param_list = 673, /* execute_param_list */ + YYSYMBOL_execute_param_clause = 674, /* execute_param_clause */ + YYSYMBOL_ExplainStmt = 675, /* ExplainStmt */ + YYSYMBOL_opt_verbose = 676, /* opt_verbose */ + YYSYMBOL_explain_option_arg = 677, /* explain_option_arg */ + YYSYMBOL_ExplainableStmt = 678, /* ExplainableStmt */ + YYSYMBOL_NonReservedWord = 679, /* NonReservedWord */ + YYSYMBOL_NonReservedWord_or_Sconst = 680, /* NonReservedWord_or_Sconst */ + YYSYMBOL_explain_option_list = 681, /* explain_option_list */ + YYSYMBOL_analyze_keyword = 682, /* analyze_keyword */ + YYSYMBOL_opt_boolean_or_string = 683, /* opt_boolean_or_string */ + YYSYMBOL_explain_option_elem = 684, /* explain_option_elem */ + YYSYMBOL_explain_option_name = 685, /* explain_option_name */ + YYSYMBOL_ExportStmt = 686, /* ExportStmt */ + YYSYMBOL_ImportStmt = 687, /* ImportStmt */ + YYSYMBOL_IndexStmt = 688, /* IndexStmt */ + YYSYMBOL_access_method = 689, /* access_method */ + YYSYMBOL_access_method_clause = 690, /* access_method_clause */ + YYSYMBOL_opt_concurrently = 691, /* opt_concurrently */ + YYSYMBOL_opt_index_name = 692, /* opt_index_name */ + YYSYMBOL_opt_reloptions = 693, /* opt_reloptions */ + YYSYMBOL_opt_unique = 694, /* opt_unique */ + YYSYMBOL_InsertStmt = 695, /* InsertStmt */ + YYSYMBOL_insert_rest = 696, /* insert_rest */ + YYSYMBOL_insert_target = 697, /* insert_target */ + YYSYMBOL_opt_by_name_or_position = 698, /* opt_by_name_or_position */ + YYSYMBOL_opt_conf_expr = 699, /* opt_conf_expr */ + YYSYMBOL_opt_with_clause = 700, /* opt_with_clause */ + YYSYMBOL_insert_column_item = 701, /* insert_column_item */ + YYSYMBOL_set_clause = 702, /* set_clause */ + YYSYMBOL_opt_or_action = 703, /* opt_or_action */ + YYSYMBOL_opt_on_conflict = 704, /* opt_on_conflict */ + YYSYMBOL_index_elem = 705, /* index_elem */ + YYSYMBOL_returning_clause = 706, /* returning_clause */ + YYSYMBOL_override_kind = 707, /* override_kind */ + YYSYMBOL_set_target_list = 708, /* set_target_list */ + YYSYMBOL_opt_collate = 709, /* opt_collate */ + YYSYMBOL_opt_class = 710, /* opt_class */ + YYSYMBOL_insert_column_list = 711, /* insert_column_list */ + YYSYMBOL_set_clause_list = 712, /* set_clause_list */ + YYSYMBOL_set_clause_list_opt_comma = 713, /* set_clause_list_opt_comma */ + YYSYMBOL_index_params = 714, /* index_params */ + YYSYMBOL_set_target = 715, /* set_target */ + YYSYMBOL_LoadStmt = 716, /* LoadStmt */ + YYSYMBOL_file_name = 717, /* file_name */ + YYSYMBOL_repo_path = 718, /* repo_path */ + YYSYMBOL_PragmaStmt = 719, /* PragmaStmt */ + YYSYMBOL_PrepareStmt = 720, /* PrepareStmt */ + YYSYMBOL_prep_type_clause = 721, /* prep_type_clause */ + YYSYMBOL_PreparableStmt = 722, /* PreparableStmt */ + YYSYMBOL_RenameStmt = 723, /* RenameStmt */ + YYSYMBOL_opt_column = 724, /* opt_column */ + YYSYMBOL_SelectStmt = 725, /* SelectStmt */ + YYSYMBOL_select_with_parens = 726, /* select_with_parens */ + YYSYMBOL_select_no_parens = 727, /* select_no_parens */ + YYSYMBOL_select_clause = 728, /* select_clause */ + YYSYMBOL_opt_select = 729, /* opt_select */ + YYSYMBOL_simple_select = 730, /* simple_select */ + YYSYMBOL_value_or_values = 731, /* value_or_values */ + YYSYMBOL_pivot_keyword = 732, /* pivot_keyword */ + YYSYMBOL_unpivot_keyword = 733, /* unpivot_keyword */ + YYSYMBOL_pivot_column_entry = 734, /* pivot_column_entry */ + YYSYMBOL_pivot_column_list_internal = 735, /* pivot_column_list_internal */ + YYSYMBOL_pivot_column_list = 736, /* pivot_column_list */ + YYSYMBOL_with_clause = 737, /* with_clause */ + YYSYMBOL_cte_list = 738, /* cte_list */ + YYSYMBOL_common_table_expr = 739, /* common_table_expr */ + YYSYMBOL_opt_materialized = 740, /* opt_materialized */ + YYSYMBOL_into_clause = 741, /* into_clause */ + YYSYMBOL_OptTempTableName = 742, /* OptTempTableName */ + YYSYMBOL_opt_table = 743, /* opt_table */ + YYSYMBOL_all_or_distinct = 744, /* all_or_distinct */ + YYSYMBOL_by_name = 745, /* by_name */ + YYSYMBOL_distinct_clause = 746, /* distinct_clause */ + YYSYMBOL_opt_all_clause = 747, /* opt_all_clause */ + YYSYMBOL_opt_ignore_nulls = 748, /* opt_ignore_nulls */ + YYSYMBOL_opt_sort_clause = 749, /* opt_sort_clause */ + YYSYMBOL_sort_clause = 750, /* sort_clause */ + YYSYMBOL_sortby_list = 751, /* sortby_list */ + YYSYMBOL_sortby = 752, /* sortby */ + YYSYMBOL_opt_asc_desc = 753, /* opt_asc_desc */ + YYSYMBOL_opt_nulls_order = 754, /* opt_nulls_order */ + YYSYMBOL_select_limit = 755, /* select_limit */ + YYSYMBOL_opt_select_limit = 756, /* opt_select_limit */ + YYSYMBOL_limit_clause = 757, /* limit_clause */ + YYSYMBOL_offset_clause = 758, /* offset_clause */ + YYSYMBOL_sample_count = 759, /* sample_count */ + YYSYMBOL_sample_clause = 760, /* sample_clause */ + YYSYMBOL_opt_sample_func = 761, /* opt_sample_func */ + YYSYMBOL_tablesample_entry = 762, /* tablesample_entry */ + YYSYMBOL_tablesample_clause = 763, /* tablesample_clause */ + YYSYMBOL_opt_tablesample_clause = 764, /* opt_tablesample_clause */ + YYSYMBOL_opt_repeatable_clause = 765, /* opt_repeatable_clause */ + YYSYMBOL_select_limit_value = 766, /* select_limit_value */ + YYSYMBOL_select_offset_value = 767, /* select_offset_value */ + YYSYMBOL_select_fetch_first_value = 768, /* select_fetch_first_value */ + YYSYMBOL_I_or_F_const = 769, /* I_or_F_const */ + YYSYMBOL_row_or_rows = 770, /* row_or_rows */ + YYSYMBOL_first_or_next = 771, /* first_or_next */ + YYSYMBOL_group_clause = 772, /* group_clause */ + YYSYMBOL_group_by_list = 773, /* group_by_list */ + YYSYMBOL_group_by_list_opt_comma = 774, /* group_by_list_opt_comma */ + YYSYMBOL_group_by_item = 775, /* group_by_item */ + YYSYMBOL_empty_grouping_set = 776, /* empty_grouping_set */ + YYSYMBOL_rollup_clause = 777, /* rollup_clause */ + YYSYMBOL_cube_clause = 778, /* cube_clause */ + YYSYMBOL_grouping_sets_clause = 779, /* grouping_sets_clause */ + YYSYMBOL_grouping_or_grouping_id = 780, /* grouping_or_grouping_id */ + YYSYMBOL_having_clause = 781, /* having_clause */ + YYSYMBOL_qualify_clause = 782, /* qualify_clause */ + YYSYMBOL_for_locking_clause = 783, /* for_locking_clause */ + YYSYMBOL_opt_for_locking_clause = 784, /* opt_for_locking_clause */ + YYSYMBOL_for_locking_items = 785, /* for_locking_items */ + YYSYMBOL_for_locking_item = 786, /* for_locking_item */ + YYSYMBOL_for_locking_strength = 787, /* for_locking_strength */ + YYSYMBOL_locked_rels_list = 788, /* locked_rels_list */ + YYSYMBOL_opt_nowait_or_skip = 789, /* opt_nowait_or_skip */ + YYSYMBOL_values_clause = 790, /* values_clause */ + YYSYMBOL_values_clause_opt_comma = 791, /* values_clause_opt_comma */ + YYSYMBOL_from_clause = 792, /* from_clause */ + YYSYMBOL_from_list = 793, /* from_list */ + YYSYMBOL_from_list_opt_comma = 794, /* from_list_opt_comma */ + YYSYMBOL_table_ref = 795, /* table_ref */ + YYSYMBOL_opt_pivot_group_by = 796, /* opt_pivot_group_by */ + YYSYMBOL_opt_include_nulls = 797, /* opt_include_nulls */ + YYSYMBOL_single_pivot_value = 798, /* single_pivot_value */ + YYSYMBOL_pivot_header = 799, /* pivot_header */ + YYSYMBOL_pivot_value = 800, /* pivot_value */ + YYSYMBOL_pivot_value_list = 801, /* pivot_value_list */ + YYSYMBOL_unpivot_header = 802, /* unpivot_header */ + YYSYMBOL_unpivot_value = 803, /* unpivot_value */ + YYSYMBOL_unpivot_value_list = 804, /* unpivot_value_list */ + YYSYMBOL_joined_table = 805, /* joined_table */ + YYSYMBOL_alias_clause = 806, /* alias_clause */ + YYSYMBOL_opt_alias_clause = 807, /* opt_alias_clause */ + YYSYMBOL_func_alias_clause = 808, /* func_alias_clause */ + YYSYMBOL_join_type = 809, /* join_type */ + YYSYMBOL_join_outer = 810, /* join_outer */ + YYSYMBOL_join_qual = 811, /* join_qual */ + YYSYMBOL_relation_expr = 812, /* relation_expr */ + YYSYMBOL_func_table = 813, /* func_table */ + YYSYMBOL_rowsfrom_item = 814, /* rowsfrom_item */ + YYSYMBOL_rowsfrom_list = 815, /* rowsfrom_list */ + YYSYMBOL_opt_col_def_list = 816, /* opt_col_def_list */ + YYSYMBOL_opt_ordinality = 817, /* opt_ordinality */ + YYSYMBOL_where_clause = 818, /* where_clause */ + YYSYMBOL_TableFuncElementList = 819, /* TableFuncElementList */ + YYSYMBOL_TableFuncElement = 820, /* TableFuncElement */ + YYSYMBOL_opt_collate_clause = 821, /* opt_collate_clause */ + YYSYMBOL_colid_type_list = 822, /* colid_type_list */ + YYSYMBOL_RowOrStruct = 823, /* RowOrStruct */ + YYSYMBOL_opt_Typename = 824, /* opt_Typename */ + YYSYMBOL_Typename = 825, /* Typename */ + YYSYMBOL_qualified_typename = 826, /* qualified_typename */ + YYSYMBOL_opt_array_bounds = 827, /* opt_array_bounds */ + YYSYMBOL_SimpleTypename = 828, /* SimpleTypename */ + YYSYMBOL_ConstTypename = 829, /* ConstTypename */ + YYSYMBOL_GenericType = 830, /* GenericType */ + YYSYMBOL_opt_type_modifiers = 831, /* opt_type_modifiers */ + YYSYMBOL_Numeric = 832, /* Numeric */ + YYSYMBOL_opt_float = 833, /* opt_float */ + YYSYMBOL_Bit = 834, /* Bit */ + YYSYMBOL_ConstBit = 835, /* ConstBit */ + YYSYMBOL_BitWithLength = 836, /* BitWithLength */ + YYSYMBOL_BitWithoutLength = 837, /* BitWithoutLength */ + YYSYMBOL_Character = 838, /* Character */ + YYSYMBOL_ConstCharacter = 839, /* ConstCharacter */ + YYSYMBOL_CharacterWithLength = 840, /* CharacterWithLength */ + YYSYMBOL_CharacterWithoutLength = 841, /* CharacterWithoutLength */ + YYSYMBOL_character = 842, /* character */ + YYSYMBOL_opt_varying = 843, /* opt_varying */ + YYSYMBOL_ConstDatetime = 844, /* ConstDatetime */ + YYSYMBOL_ConstInterval = 845, /* ConstInterval */ + YYSYMBOL_opt_timezone = 846, /* opt_timezone */ + YYSYMBOL_year_keyword = 847, /* year_keyword */ + YYSYMBOL_month_keyword = 848, /* month_keyword */ + YYSYMBOL_day_keyword = 849, /* day_keyword */ + YYSYMBOL_hour_keyword = 850, /* hour_keyword */ + YYSYMBOL_minute_keyword = 851, /* minute_keyword */ + YYSYMBOL_second_keyword = 852, /* second_keyword */ + YYSYMBOL_millisecond_keyword = 853, /* millisecond_keyword */ + YYSYMBOL_microsecond_keyword = 854, /* microsecond_keyword */ + YYSYMBOL_week_keyword = 855, /* week_keyword */ + YYSYMBOL_decade_keyword = 856, /* decade_keyword */ + YYSYMBOL_century_keyword = 857, /* century_keyword */ + YYSYMBOL_millennium_keyword = 858, /* millennium_keyword */ + YYSYMBOL_opt_interval = 859, /* opt_interval */ + YYSYMBOL_a_expr = 860, /* a_expr */ + YYSYMBOL_b_expr = 861, /* b_expr */ + YYSYMBOL_c_expr = 862, /* c_expr */ + YYSYMBOL_d_expr = 863, /* d_expr */ + YYSYMBOL_indirection_expr = 864, /* indirection_expr */ + YYSYMBOL_struct_expr = 865, /* struct_expr */ + YYSYMBOL_func_application = 866, /* func_application */ + YYSYMBOL_func_expr = 867, /* func_expr */ + YYSYMBOL_func_expr_windowless = 868, /* func_expr_windowless */ + YYSYMBOL_func_expr_common_subexpr = 869, /* func_expr_common_subexpr */ + YYSYMBOL_list_comprehension = 870, /* list_comprehension */ + YYSYMBOL_within_group_clause = 871, /* within_group_clause */ + YYSYMBOL_filter_clause = 872, /* filter_clause */ + YYSYMBOL_export_clause = 873, /* export_clause */ + YYSYMBOL_window_clause = 874, /* window_clause */ + YYSYMBOL_window_definition_list = 875, /* window_definition_list */ + YYSYMBOL_window_definition = 876, /* window_definition */ + YYSYMBOL_over_clause = 877, /* over_clause */ + YYSYMBOL_window_specification = 878, /* window_specification */ + YYSYMBOL_opt_existing_window_name = 879, /* opt_existing_window_name */ + YYSYMBOL_opt_partition_clause = 880, /* opt_partition_clause */ + YYSYMBOL_opt_frame_clause = 881, /* opt_frame_clause */ + YYSYMBOL_frame_extent = 882, /* frame_extent */ + YYSYMBOL_frame_bound = 883, /* frame_bound */ + YYSYMBOL_opt_window_exclusion_clause = 884, /* opt_window_exclusion_clause */ + YYSYMBOL_qualified_row = 885, /* qualified_row */ + YYSYMBOL_row = 886, /* row */ + YYSYMBOL_dict_arg = 887, /* dict_arg */ + YYSYMBOL_dict_arguments = 888, /* dict_arguments */ + YYSYMBOL_dict_arguments_opt_comma = 889, /* dict_arguments_opt_comma */ + YYSYMBOL_map_arg = 890, /* map_arg */ + YYSYMBOL_map_arguments = 891, /* map_arguments */ + YYSYMBOL_map_arguments_opt_comma = 892, /* map_arguments_opt_comma */ + YYSYMBOL_opt_map_arguments_opt_comma = 893, /* opt_map_arguments_opt_comma */ + YYSYMBOL_sub_type = 894, /* sub_type */ + YYSYMBOL_all_Op = 895, /* all_Op */ + YYSYMBOL_MathOp = 896, /* MathOp */ + YYSYMBOL_qual_Op = 897, /* qual_Op */ + YYSYMBOL_qual_all_Op = 898, /* qual_all_Op */ + YYSYMBOL_subquery_Op = 899, /* subquery_Op */ + YYSYMBOL_any_operator = 900, /* any_operator */ + YYSYMBOL_c_expr_list = 901, /* c_expr_list */ + YYSYMBOL_c_expr_list_opt_comma = 902, /* c_expr_list_opt_comma */ + YYSYMBOL_expr_list = 903, /* expr_list */ + YYSYMBOL_expr_list_opt_comma = 904, /* expr_list_opt_comma */ + YYSYMBOL_opt_expr_list_opt_comma = 905, /* opt_expr_list_opt_comma */ + YYSYMBOL_func_arg_list = 906, /* func_arg_list */ + YYSYMBOL_func_arg_expr = 907, /* func_arg_expr */ + YYSYMBOL_type_list = 908, /* type_list */ + YYSYMBOL_extract_list = 909, /* extract_list */ + YYSYMBOL_extract_arg = 910, /* extract_arg */ + YYSYMBOL_overlay_list = 911, /* overlay_list */ + YYSYMBOL_overlay_placing = 912, /* overlay_placing */ + YYSYMBOL_position_list = 913, /* position_list */ + YYSYMBOL_substr_list = 914, /* substr_list */ + YYSYMBOL_substr_from = 915, /* substr_from */ + YYSYMBOL_substr_for = 916, /* substr_for */ + YYSYMBOL_trim_list = 917, /* trim_list */ + YYSYMBOL_in_expr = 918, /* in_expr */ + YYSYMBOL_case_expr = 919, /* case_expr */ + YYSYMBOL_when_clause_list = 920, /* when_clause_list */ + YYSYMBOL_when_clause = 921, /* when_clause */ + YYSYMBOL_case_default = 922, /* case_default */ + YYSYMBOL_case_arg = 923, /* case_arg */ + YYSYMBOL_columnref = 924, /* columnref */ + YYSYMBOL_opt_slice_bound = 925, /* opt_slice_bound */ + YYSYMBOL_opt_indirection = 926, /* opt_indirection */ + YYSYMBOL_opt_func_arguments = 927, /* opt_func_arguments */ + YYSYMBOL_extended_indirection_el = 928, /* extended_indirection_el */ + YYSYMBOL_opt_extended_indirection = 929, /* opt_extended_indirection */ + YYSYMBOL_opt_asymmetric = 930, /* opt_asymmetric */ + YYSYMBOL_opt_target_list_opt_comma = 931, /* opt_target_list_opt_comma */ + YYSYMBOL_target_list = 932, /* target_list */ + YYSYMBOL_target_list_opt_comma = 933, /* target_list_opt_comma */ + YYSYMBOL_target_el = 934, /* target_el */ + YYSYMBOL_except_list = 935, /* except_list */ + YYSYMBOL_opt_except_list = 936, /* opt_except_list */ + YYSYMBOL_replace_list_el = 937, /* replace_list_el */ + YYSYMBOL_replace_list = 938, /* replace_list */ + YYSYMBOL_replace_list_opt_comma = 939, /* replace_list_opt_comma */ + YYSYMBOL_opt_replace_list = 940, /* opt_replace_list */ + YYSYMBOL_qualified_name_list = 941, /* qualified_name_list */ + YYSYMBOL_name_list = 942, /* name_list */ + YYSYMBOL_name_list_opt_comma = 943, /* name_list_opt_comma */ + YYSYMBOL_name_list_opt_comma_opt_bracket = 944, /* name_list_opt_comma_opt_bracket */ + YYSYMBOL_name = 945, /* name */ + YYSYMBOL_func_name = 946, /* func_name */ + YYSYMBOL_AexprConst = 947, /* AexprConst */ + YYSYMBOL_Iconst = 948, /* Iconst */ + YYSYMBOL_type_function_name = 949, /* type_function_name */ + YYSYMBOL_function_name_token = 950, /* function_name_token */ + YYSYMBOL_type_name_token = 951, /* type_name_token */ + YYSYMBOL_any_name = 952, /* any_name */ + YYSYMBOL_attrs = 953, /* attrs */ + YYSYMBOL_opt_name_list = 954, /* opt_name_list */ + YYSYMBOL_param_name = 955, /* param_name */ + YYSYMBOL_ColLabelOrString = 956, /* ColLabelOrString */ + YYSYMBOL_TransactionStmt = 957, /* TransactionStmt */ + YYSYMBOL_opt_transaction = 958, /* opt_transaction */ + YYSYMBOL_UpdateStmt = 959, /* UpdateStmt */ + YYSYMBOL_UseStmt = 960, /* UseStmt */ + YYSYMBOL_VacuumStmt = 961, /* VacuumStmt */ + YYSYMBOL_vacuum_option_elem = 962, /* vacuum_option_elem */ + YYSYMBOL_opt_full = 963, /* opt_full */ + YYSYMBOL_vacuum_option_list = 964, /* vacuum_option_list */ + YYSYMBOL_opt_freeze = 965, /* opt_freeze */ + YYSYMBOL_VariableResetStmt = 966, /* VariableResetStmt */ + YYSYMBOL_generic_reset = 967, /* generic_reset */ + YYSYMBOL_reset_rest = 968, /* reset_rest */ + YYSYMBOL_VariableSetStmt = 969, /* VariableSetStmt */ + YYSYMBOL_set_rest = 970, /* set_rest */ + YYSYMBOL_generic_set = 971, /* generic_set */ + YYSYMBOL_var_value = 972, /* var_value */ + YYSYMBOL_zone_value = 973, /* zone_value */ + YYSYMBOL_var_list = 974, /* var_list */ + YYSYMBOL_VariableShowStmt = 975, /* VariableShowStmt */ + YYSYMBOL_describe_or_desc = 976, /* describe_or_desc */ + YYSYMBOL_show_or_describe = 977, /* show_or_describe */ + YYSYMBOL_opt_tables = 978, /* opt_tables */ + YYSYMBOL_var_name = 979, /* var_name */ + YYSYMBOL_table_id = 980, /* table_id */ + YYSYMBOL_ViewStmt = 981, /* ViewStmt */ + YYSYMBOL_opt_check_option = 982 /* opt_check_option */ +}; +typedef enum yysymbol_kind_t yysymbol_kind_t; -/* Copy the second part of user declarations. */ -/* Line 216 of yacc.c. */ -#line 1340 "third_party/libpg_query/grammar/grammar_out.cpp" #ifdef short # undef short #endif -#ifdef YYTYPE_UINT8 -typedef YYTYPE_UINT8 yytype_uint8; -#else -typedef unsigned char yytype_uint8; +/* On compilers that do not define __PTRDIFF_MAX__ etc., make sure + and (if available) are included + so that the code can choose integer types of a good width. */ + +#ifndef __PTRDIFF_MAX__ +# include /* INFRINGES ON USER NAME SPACE */ +# if defined __STDC_VERSION__ && 199901 <= __STDC_VERSION__ +# include /* INFRINGES ON USER NAME SPACE */ +# define YY_STDINT_H +# endif #endif -#ifdef YYTYPE_INT8 -typedef YYTYPE_INT8 yytype_int8; -#elif (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +/* Narrow types that promote to a signed type and that can represent a + signed or unsigned integer of at least N bits. In tables they can + save space and decrease cache pressure. Promoting to a signed type + helps avoid bugs in integer arithmetic. */ + +#ifdef __INT_LEAST8_MAX__ +typedef __INT_LEAST8_TYPE__ yytype_int8; +#elif defined YY_STDINT_H +typedef int_least8_t yytype_int8; +#else typedef signed char yytype_int8; +#endif + +#ifdef __INT_LEAST16_MAX__ +typedef __INT_LEAST16_TYPE__ yytype_int16; +#elif defined YY_STDINT_H +typedef int_least16_t yytype_int16; #else -typedef short int yytype_int8; +typedef short yytype_int16; +#endif + +/* Work around bug in HP-UX 11.23, which defines these macros + incorrectly for preprocessor constants. This workaround can likely + be removed in 2023, as HPE has promised support for HP-UX 11.23 + (aka HP-UX 11i v2) only through the end of 2022; see Table 2 of + . */ +#ifdef __hpux +# undef UINT_LEAST8_MAX +# undef UINT_LEAST16_MAX +# define UINT_LEAST8_MAX 255 +# define UINT_LEAST16_MAX 65535 #endif -#ifdef YYTYPE_UINT16 -typedef YYTYPE_UINT16 yytype_uint16; +#if defined __UINT_LEAST8_MAX__ && __UINT_LEAST8_MAX__ <= __INT_MAX__ +typedef __UINT_LEAST8_TYPE__ yytype_uint8; +#elif (!defined __UINT_LEAST8_MAX__ && defined YY_STDINT_H \ + && UINT_LEAST8_MAX <= INT_MAX) +typedef uint_least8_t yytype_uint8; +#elif !defined __UINT_LEAST8_MAX__ && UCHAR_MAX <= INT_MAX +typedef unsigned char yytype_uint8; #else -typedef unsigned short int yytype_uint16; +typedef short yytype_uint8; #endif -#ifdef YYTYPE_INT16 -typedef YYTYPE_INT16 yytype_int16; +#if defined __UINT_LEAST16_MAX__ && __UINT_LEAST16_MAX__ <= __INT_MAX__ +typedef __UINT_LEAST16_TYPE__ yytype_uint16; +#elif (!defined __UINT_LEAST16_MAX__ && defined YY_STDINT_H \ + && UINT_LEAST16_MAX <= INT_MAX) +typedef uint_least16_t yytype_uint16; +#elif !defined __UINT_LEAST16_MAX__ && USHRT_MAX <= INT_MAX +typedef unsigned short yytype_uint16; #else -typedef short int yytype_int16; +typedef int yytype_uint16; +#endif + +#ifndef YYPTRDIFF_T +# if defined __PTRDIFF_TYPE__ && defined __PTRDIFF_MAX__ +# define YYPTRDIFF_T __PTRDIFF_TYPE__ +# define YYPTRDIFF_MAXIMUM __PTRDIFF_MAX__ +# elif defined PTRDIFF_MAX +# ifndef ptrdiff_t +# include /* INFRINGES ON USER NAME SPACE */ +# endif +# define YYPTRDIFF_T ptrdiff_t +# define YYPTRDIFF_MAXIMUM PTRDIFF_MAX +# else +# define YYPTRDIFF_T long +# define YYPTRDIFF_MAXIMUM LONG_MAX +# endif #endif #ifndef YYSIZE_T @@ -1374,55 +1346,106 @@ typedef short int yytype_int16; # define YYSIZE_T __SIZE_TYPE__ # elif defined size_t # define YYSIZE_T size_t -# elif ! defined YYSIZE_T && (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +# elif defined __STDC_VERSION__ && 199901 <= __STDC_VERSION__ # include /* INFRINGES ON USER NAME SPACE */ # define YYSIZE_T size_t # else -# define YYSIZE_T unsigned int +# define YYSIZE_T unsigned # endif #endif -#define YYSIZE_MAXIMUM ((YYSIZE_T) -1) +#define YYSIZE_MAXIMUM \ + YY_CAST (YYPTRDIFF_T, \ + (YYPTRDIFF_MAXIMUM < YY_CAST (YYSIZE_T, -1) \ + ? YYPTRDIFF_MAXIMUM \ + : YY_CAST (YYSIZE_T, -1))) + +#define YYSIZEOF(X) YY_CAST (YYPTRDIFF_T, sizeof (X)) + + +/* Stored state numbers (used for stacks). */ +typedef yytype_int16 yy_state_t; + +/* State numbers in computations. */ +typedef int yy_state_fast_t; #ifndef YY_ # if defined YYENABLE_NLS && YYENABLE_NLS # if ENABLE_NLS # include /* INFRINGES ON USER NAME SPACE */ -# define YY_(msgid) dgettext ("bison-runtime", msgid) +# define YY_(Msgid) dgettext ("bison-runtime", Msgid) # endif # endif # ifndef YY_ -# define YY_(msgid) msgid +# define YY_(Msgid) Msgid +# endif +#endif + + +#ifndef YY_ATTRIBUTE_PURE +# if defined __GNUC__ && 2 < __GNUC__ + (96 <= __GNUC_MINOR__) +# define YY_ATTRIBUTE_PURE __attribute__ ((__pure__)) +# else +# define YY_ATTRIBUTE_PURE +# endif +#endif + +#ifndef YY_ATTRIBUTE_UNUSED +# if defined __GNUC__ && 2 < __GNUC__ + (7 <= __GNUC_MINOR__) +# define YY_ATTRIBUTE_UNUSED __attribute__ ((__unused__)) +# else +# define YY_ATTRIBUTE_UNUSED # endif #endif /* Suppress unused-variable warnings by "using" E. */ #if ! defined lint || defined __GNUC__ -# define YYUSE(e) ((void) (e)) +# define YY_USE(E) ((void) (E)) #else -# define YYUSE(e) /* empty */ +# define YY_USE(E) /* empty */ #endif -/* Identity function, used to suppress warnings about constant conditions. */ -#ifndef lint -# define YYID(n) (n) -#else -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static int -YYID (int i) +/* Suppress an incorrect diagnostic about yylval being uninitialized. */ +#if defined __GNUC__ && ! defined __ICC && 406 <= __GNUC__ * 100 + __GNUC_MINOR__ +# if __GNUC__ * 100 + __GNUC_MINOR__ < 407 +# define YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN \ + _Pragma ("GCC diagnostic push") \ + _Pragma ("GCC diagnostic ignored \"-Wuninitialized\"") +# else +# define YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN \ + _Pragma ("GCC diagnostic push") \ + _Pragma ("GCC diagnostic ignored \"-Wuninitialized\"") \ + _Pragma ("GCC diagnostic ignored \"-Wmaybe-uninitialized\"") +# endif +# define YY_IGNORE_MAYBE_UNINITIALIZED_END \ + _Pragma ("GCC diagnostic pop") #else -static int -YYID (i) - int i; +# define YY_INITIAL_VALUE(Value) Value #endif -{ - return i; -} +#ifndef YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN +# define YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN +# define YY_IGNORE_MAYBE_UNINITIALIZED_END +#endif +#ifndef YY_INITIAL_VALUE +# define YY_INITIAL_VALUE(Value) /* Nothing. */ +#endif + +#if defined __cplusplus && defined __GNUC__ && ! defined __ICC && 6 <= __GNUC__ +# define YY_IGNORE_USELESS_CAST_BEGIN \ + _Pragma ("GCC diagnostic push") \ + _Pragma ("GCC diagnostic ignored \"-Wuseless-cast\"") +# define YY_IGNORE_USELESS_CAST_END \ + _Pragma ("GCC diagnostic pop") +#endif +#ifndef YY_IGNORE_USELESS_CAST_BEGIN +# define YY_IGNORE_USELESS_CAST_BEGIN +# define YY_IGNORE_USELESS_CAST_END #endif -#if ! defined yyoverflow || YYERROR_VERBOSE + +#define YY_ASSERT(E) ((void) (0 && (E))) + +#if !defined yyoverflow /* The parser invokes alloca or malloc; define the necessary symbols. */ @@ -1439,11 +1462,11 @@ YYID (i) # define alloca _alloca # else # define YYSTACK_ALLOC alloca -# if ! defined _ALLOCA_H && ! defined _STDLIB_H && (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +# if ! defined _ALLOCA_H && ! defined EXIT_SUCCESS # include /* INFRINGES ON USER NAME SPACE */ -# ifndef _STDLIB_H -# define _STDLIB_H 1 + /* Use EXIT_SUCCESS as a witness for stdlib.h. */ +# ifndef EXIT_SUCCESS +# define EXIT_SUCCESS 0 # endif # endif # endif @@ -1451,8 +1474,8 @@ YYID (i) # endif # ifdef YYSTACK_ALLOC - /* Pacify GCC's `empty if-body' warning. */ -# define YYSTACK_FREE(Ptr) do { /* empty */; } while (YYID (0)) + /* Pacify GCC's 'empty if-body' warning. */ +# define YYSTACK_FREE(Ptr) do { /* empty */; } while (0) # ifndef YYSTACK_ALLOC_MAXIMUM /* The OS might guarantee only one guard page at the bottom of the stack, and a page size can be as small as 4096 bytes. So we cannot safely @@ -1466,127 +1489,134 @@ YYID (i) # ifndef YYSTACK_ALLOC_MAXIMUM # define YYSTACK_ALLOC_MAXIMUM YYSIZE_MAXIMUM # endif -# if (defined __cplusplus && ! defined _STDLIB_H \ +# if (defined __cplusplus && ! defined EXIT_SUCCESS \ && ! ((defined YYMALLOC || defined malloc) \ - && (defined YYFREE || defined free))) + && (defined YYFREE || defined free))) # include /* INFRINGES ON USER NAME SPACE */ -# ifndef _STDLIB_H -# define _STDLIB_H 1 +# ifndef EXIT_SUCCESS +# define EXIT_SUCCESS 0 # endif # endif # ifndef YYMALLOC # define YYMALLOC malloc -# if ! defined malloc && ! defined _STDLIB_H && (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +# if ! defined malloc && ! defined EXIT_SUCCESS void *malloc (YYSIZE_T); /* INFRINGES ON USER NAME SPACE */ # endif # endif # ifndef YYFREE # define YYFREE free -# if ! defined free && ! defined _STDLIB_H && (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +# if ! defined free && ! defined EXIT_SUCCESS void free (void *); /* INFRINGES ON USER NAME SPACE */ # endif # endif # endif -#endif /* ! defined yyoverflow || YYERROR_VERBOSE */ - +#endif /* !defined yyoverflow */ #if (! defined yyoverflow \ && (! defined __cplusplus \ - || (defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL \ - && defined YYSTYPE_IS_TRIVIAL && YYSTYPE_IS_TRIVIAL))) + || (defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL \ + && defined YYSTYPE_IS_TRIVIAL && YYSTYPE_IS_TRIVIAL))) /* A type that is properly aligned for any stack member. */ union yyalloc { - yytype_int16 yyss; - YYSTYPE yyvs; - YYLTYPE yyls; + yy_state_t yyss_alloc; + YYSTYPE yyvs_alloc; + YYLTYPE yyls_alloc; }; /* The size of the maximum gap between one aligned stack and the next. */ -# define YYSTACK_GAP_MAXIMUM (sizeof (union yyalloc) - 1) +# define YYSTACK_GAP_MAXIMUM (YYSIZEOF (union yyalloc) - 1) /* The size of an array large to enough to hold all stacks, each with N elements. */ # define YYSTACK_BYTES(N) \ - ((N) * (sizeof (yytype_int16) + sizeof (YYSTYPE) + sizeof (YYLTYPE)) \ + ((N) * (YYSIZEOF (yy_state_t) + YYSIZEOF (YYSTYPE) \ + + YYSIZEOF (YYLTYPE)) \ + 2 * YYSTACK_GAP_MAXIMUM) -/* Copy COUNT objects from FROM to TO. The source and destination do - not overlap. */ -# ifndef YYCOPY -# if defined __GNUC__ && 1 < __GNUC__ -# define YYCOPY(To, From, Count) \ - __builtin_memcpy (To, From, (Count) * sizeof (*(From))) -# else -# define YYCOPY(To, From, Count) \ - do \ - { \ - YYSIZE_T yyi; \ - for (yyi = 0; yyi < (Count); yyi++) \ - (To)[yyi] = (From)[yyi]; \ - } \ - while (YYID (0)) -# endif -# endif +# define YYCOPY_NEEDED 1 /* Relocate STACK from its old location to the new one. The local variables YYSIZE and YYSTACKSIZE give the old and new number of elements in the stack, and YYPTR gives the new location of the stack. Advance YYPTR to a properly aligned location for the next stack. */ -# define YYSTACK_RELOCATE(Stack) \ - do \ - { \ - YYSIZE_T yynewbytes; \ - YYCOPY (&yyptr->Stack, Stack, yysize); \ - Stack = &yyptr->Stack; \ - yynewbytes = yystacksize * sizeof (*Stack) + YYSTACK_GAP_MAXIMUM; \ - yyptr += yynewbytes / sizeof (*yyptr); \ - } \ - while (YYID (0)) +# define YYSTACK_RELOCATE(Stack_alloc, Stack) \ + do \ + { \ + YYPTRDIFF_T yynewbytes; \ + YYCOPY (&yyptr->Stack_alloc, Stack, yysize); \ + Stack = &yyptr->Stack_alloc; \ + yynewbytes = yystacksize * YYSIZEOF (*Stack) + YYSTACK_GAP_MAXIMUM; \ + yyptr += yynewbytes / YYSIZEOF (*yyptr); \ + } \ + while (0) #endif +#if defined YYCOPY_NEEDED && YYCOPY_NEEDED +/* Copy COUNT objects from SRC to DST. The source and destination do + not overlap. */ +# ifndef YYCOPY +# if defined __GNUC__ && 1 < __GNUC__ +# define YYCOPY(Dst, Src, Count) \ + __builtin_memcpy (Dst, Src, YY_CAST (YYSIZE_T, (Count)) * sizeof (*(Src))) +# else +# define YYCOPY(Dst, Src, Count) \ + do \ + { \ + YYPTRDIFF_T yyi; \ + for (yyi = 0; yyi < (Count); yyi++) \ + (Dst)[yyi] = (Src)[yyi]; \ + } \ + while (0) +# endif +# endif +#endif /* !YYCOPY_NEEDED */ + /* YYFINAL -- State number of the termination state. */ -#define YYFINAL 693 +#define YYFINAL 867 /* YYLAST -- Last index in YYTABLE. */ -#define YYLAST 68980 +#define YYLAST 72555 /* YYNTOKENS -- Number of terminals. */ -#define YYNTOKENS 522 +#define YYNTOKENS 525 /* YYNNTS -- Number of nonterminals. */ -#define YYNNTS 448 +#define YYNNTS 458 /* YYNRULES -- Number of rules. */ -#define YYNRULES 2074 -/* YYNRULES -- Number of states. */ -#define YYNSTATES 3451 +#define YYNRULES 2115 +/* YYNSTATES -- Number of states. */ +#define YYNSTATES 3524 -/* YYTRANSLATE(YYLEX) -- Bison symbol number corresponding to YYLEX. */ -#define YYUNDEFTOK 2 -#define YYMAXUTOK 754 +/* YYMAXUTOK -- Last valid token kind. */ +#define YYMAXUTOK 757 -#define YYTRANSLATE(YYX) \ - ((unsigned int) (YYX) <= YYMAXUTOK ? yytranslate[YYX] : YYUNDEFTOK) -/* YYTRANSLATE[YYLEX] -- Bison symbol number corresponding to YYLEX. */ -static const yytype_uint16 yytranslate[] = +/* YYTRANSLATE(TOKEN-NUM) -- Symbol number corresponding to TOKEN-NUM + as returned by yylex, with out-of-bounds checking. */ +#define YYTRANSLATE(YYX) \ + (0 <= (YYX) && (YYX) <= YYMAXUTOK \ + ? YY_CAST (yysymbol_kind_t, yytranslate[YYX]) \ + : YYSYMBOL_YYUNDEF) + +/* YYTRANSLATE[TOKEN-NUM] -- Symbol number corresponding to TOKEN-NUM + as returned by yylex. */ +static const yytype_int16 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, 516, 517, 506, 2, 2, - 511, 512, 504, 502, 515, 503, 513, 505, 2, 2, - 2, 2, 2, 2, 2, 2, 2, 2, 521, 514, - 498, 500, 499, 518, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 519, 520, 509, 2, 2, + 514, 515, 507, 505, 518, 506, 516, 508, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 524, 517, + 501, 503, 502, 521, 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, 509, 2, 510, 507, 2, 2, 2, 2, 2, + 2, 512, 2, 513, 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, 2, 519, 2, 520, 2, 2, 2, 2, + 2, 2, 2, 522, 2, 523, 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, @@ -1649,1002 +1679,163 @@ static const yytype_uint16 yytranslate[] = 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 501, 508 + 495, 496, 497, 498, 499, 500, 504, 511 }; #if YYDEBUG -/* YYPRHS[YYN] -- Index of the first RHS symbol of rule number YYN in - YYRHS. */ -static const yytype_uint16 yyprhs[] = -{ - 0, 0, 3, 5, 9, 11, 13, 15, 17, 19, - 21, 23, 25, 27, 29, 31, 33, 35, 37, 39, - 41, 43, 45, 47, 49, 51, 53, 55, 57, 59, - 61, 63, 65, 67, 69, 71, 73, 75, 77, 79, - 81, 83, 85, 86, 91, 98, 103, 110, 115, 122, - 127, 134, 136, 139, 143, 146, 148, 152, 155, 159, - 161, 165, 168, 174, 178, 185, 190, 197, 204, 211, - 217, 223, 230, 240, 245, 251, 259, 266, 271, 280, - 285, 288, 293, 297, 304, 309, 312, 315, 318, 321, - 323, 326, 327, 329, 332, 335, 338, 340, 344, 349, - 352, 354, 355, 358, 362, 365, 369, 371, 374, 376, - 378, 380, 382, 384, 386, 388, 391, 394, 396, 398, - 400, 402, 404, 411, 418, 427, 434, 443, 450, 459, - 466, 475, 484, 495, 504, 515, 517, 518, 528, 530, - 535, 540, 548, 551, 553, 557, 560, 563, 564, 569, - 573, 574, 576, 577, 580, 584, 590, 593, 596, 597, - 606, 612, 613, 619, 625, 633, 636, 637, 639, 641, - 643, 647, 650, 651, 653, 654, 656, 660, 662, 666, - 668, 671, 673, 677, 680, 687, 696, 702, 704, 705, - 707, 711, 714, 719, 725, 731, 740, 748, 750, 751, - 755, 765, 778, 780, 784, 786, 790, 794, 795, 800, - 807, 809, 812, 814, 816, 817, 819, 822, 825, 827, - 830, 833, 835, 838, 842, 845, 848, 851, 854, 858, - 862, 866, 868, 872, 874, 875, 877, 880, 883, 886, - 889, 892, 895, 898, 901, 903, 905, 906, 909, 919, - 932, 944, 945, 948, 950, 952, 954, 956, 958, 960, - 964, 965, 967, 970, 972, 974, 977, 980, 984, 986, - 988, 991, 994, 996, 999, 1003, 1009, 1013, 1016, 1022, - 1024, 1026, 1028, 1029, 1035, 1043, 1049, 1052, 1056, 1058, - 1060, 1063, 1066, 1067, 1071, 1076, 1081, 1082, 1086, 1089, - 1090, 1094, 1096, 1098, 1100, 1102, 1104, 1106, 1108, 1110, - 1112, 1114, 1118, 1122, 1124, 1127, 1130, 1133, 1136, 1139, - 1142, 1143, 1147, 1151, 1155, 1156, 1158, 1161, 1163, 1166, - 1169, 1172, 1175, 1179, 1184, 1186, 1190, 1192, 1194, 1196, - 1198, 1202, 1204, 1207, 1208, 1210, 1213, 1214, 1216, 1220, - 1221, 1224, 1225, 1229, 1233, 1235, 1241, 1245, 1247, 1251, - 1253, 1256, 1258, 1263, 1269, 1275, 1282, 1286, 1294, 1299, - 1311, 1313, 1317, 1320, 1323, 1326, 1327, 1331, 1333, 1335, - 1338, 1341, 1344, 1347, 1349, 1350, 1352, 1355, 1362, 1367, - 1374, 1379, 1386, 1395, 1397, 1399, 1401, 1403, 1406, 1408, - 1411, 1413, 1416, 1418, 1420, 1422, 1424, 1428, 1432, 1436, - 1440, 1442, 1445, 1448, 1450, 1454, 1456, 1458, 1460, 1464, - 1466, 1468, 1469, 1471, 1473, 1475, 1484, 1496, 1507, 1515, - 1526, 1536, 1538, 1540, 1543, 1547, 1556, 1568, 1578, 1586, - 1587, 1591, 1595, 1597, 1599, 1603, 1604, 1606, 1610, 1612, - 1613, 1615, 1616, 1618, 1619, 1621, 1625, 1627, 1629, 1631, - 1635, 1637, 1638, 1641, 1644, 1645, 1648, 1649, 1651, 1652, - 1654, 1656, 1658, 1662, 1666, 1668, 1670, 1674, 1678, 1682, - 1686, 1690, 1694, 1699, 1703, 1706, 1708, 1710, 1712, 1714, - 1716, 1720, 1722, 1724, 1728, 1732, 1734, 1737, 1742, 1747, - 1750, 1754, 1760, 1766, 1768, 1770, 1774, 1775, 1787, 1799, - 1810, 1823, 1825, 1828, 1834, 1839, 1844, 1849, 1854, 1862, - 1868, 1873, 1881, 1888, 1898, 1908, 1913, 1915, 1917, 1919, - 1921, 1923, 1925, 1927, 1933, 1935, 1937, 1941, 1943, 1946, - 1949, 1952, 1956, 1958, 1962, 1970, 1972, 1975, 1976, 1979, - 1980, 1984, 1988, 1993, 1998, 2003, 2008, 2012, 2015, 2017, - 2019, 2020, 2022, 2024, 2025, 2028, 2030, 2036, 2038, 2039, - 2042, 2045, 2046, 2048, 2049, 2053, 2059, 2061, 2065, 2070, - 2074, 2076, 2078, 2079, 2082, 2085, 2086, 2089, 2092, 2094, - 2096, 2098, 2099, 2102, 2107, 2113, 2118, 2121, 2125, 2128, - 2131, 2134, 2137, 2139, 2142, 2146, 2147, 2149, 2150, 2156, - 2158, 2163, 2170, 2173, 2175, 2176, 2181, 2182, 2184, 2186, - 2189, 2192, 2195, 2197, 2199, 2202, 2205, 2207, 2209, 2211, - 2213, 2215, 2217, 2221, 2225, 2226, 2228, 2232, 2234, 2237, - 2239, 2241, 2243, 2245, 2247, 2250, 2255, 2260, 2266, 2268, - 2270, 2273, 2274, 2277, 2278, 2280, 2284, 2286, 2287, 2289, - 2292, 2296, 2299, 2304, 2307, 2311, 2314, 2315, 2317, 2320, - 2321, 2326, 2332, 2334, 2337, 2340, 2341, 2343, 2347, 2349, - 2352, 2356, 2360, 2364, 2368, 2372, 2376, 2378, 2383, 2393, - 2403, 2407, 2408, 2411, 2414, 2415, 2421, 2425, 2427, 2431, - 2437, 2441, 2443, 2446, 2448, 2452, 2458, 2460, 2463, 2467, - 2472, 2478, 2483, 2489, 2494, 2501, 2507, 2512, 2518, 2524, - 2530, 2533, 2538, 2540, 2542, 2543, 2545, 2550, 2556, 2561, - 2562, 2565, 2568, 2571, 2573, 2575, 2577, 2579, 2580, 2585, - 2588, 2590, 2593, 2596, 2601, 2604, 2611, 2614, 2616, 2620, - 2625, 2626, 2629, 2630, 2633, 2634, 2636, 2640, 2644, 2647, - 2648, 2651, 2656, 2658, 2660, 2662, 2663, 2666, 2670, 2676, - 2683, 2686, 2690, 2692, 2698, 2704, 2710, 2714, 2718, 2722, - 2727, 2728, 2730, 2732, 2734, 2736, 2738, 2741, 2746, 2748, - 2750, 2752, 2754, 2757, 2761, 2762, 2764, 2766, 2768, 2770, - 2772, 2775, 2778, 2781, 2784, 2787, 2789, 2793, 2794, 2796, - 2798, 2800, 2802, 2808, 2811, 2813, 2815, 2817, 2819, 2824, - 2826, 2829, 2832, 2834, 2838, 2842, 2845, 2847, 2848, 2854, - 2857, 2863, 2866, 2868, 2872, 2876, 2877, 2879, 2881, 2883, - 2885, 2887, 2889, 2891, 2893, 2895, 2897, 2899, 2901, 2903, - 2905, 2907, 2909, 2911, 2913, 2915, 2917, 2919, 2921, 2923, - 2925, 2927, 2929, 2931, 2933, 2935, 2937, 2939, 2941, 2943, - 2945, 2947, 2949, 2953, 2957, 2961, 2965, 2969, 2973, 2977, - 2978, 2980, 2984, 2988, 2994, 2997, 3000, 3004, 3008, 3012, - 3016, 3020, 3024, 3028, 3032, 3036, 3040, 3044, 3048, 3052, - 3056, 3060, 3063, 3066, 3070, 3074, 3077, 3080, 3084, 3088, - 3094, 3099, 3106, 3110, 3116, 3121, 3128, 3133, 3140, 3146, - 3154, 3158, 3161, 3166, 3170, 3173, 3177, 3181, 3185, 3189, - 3194, 3198, 3203, 3207, 3212, 3218, 3225, 3232, 3240, 3247, - 3255, 3262, 3270, 3274, 3279, 3284, 3291, 3293, 3298, 3302, - 3308, 3310, 3314, 3317, 3320, 3324, 3328, 3332, 3336, 3340, - 3344, 3348, 3352, 3356, 3360, 3364, 3368, 3372, 3376, 3380, - 3383, 3386, 3392, 3399, 3406, 3414, 3416, 3418, 3421, 3423, - 3425, 3428, 3431, 3435, 3437, 3440, 3445, 3447, 3449, 3452, - 3455, 3460, 3462, 3464, 3468, 3470, 3475, 3477, 3481, 3485, - 3492, 3500, 3510, 3518, 3526, 3532, 3534, 3536, 3538, 3544, - 3551, 3558, 3563, 3568, 3573, 3578, 3585, 3591, 3597, 3603, - 3608, 3615, 3620, 3628, 3638, 3644, 3645, 3651, 3656, 3657, - 3659, 3660, 3663, 3664, 3666, 3670, 3674, 3677, 3680, 3681, - 3688, 3690, 3691, 3695, 3696, 3700, 3704, 3708, 3709, 3711, - 3716, 3719, 3722, 3725, 3728, 3731, 3735, 3738, 3741, 3745, - 3746, 3751, 3755, 3757, 3763, 3767, 3769, 3773, 3775, 3778, - 3782, 3784, 3788, 3790, 3793, 3795, 3796, 3798, 3800, 3802, - 3804, 3806, 3808, 3810, 3812, 3814, 3816, 3818, 3820, 3822, - 3824, 3826, 3828, 3830, 3832, 3834, 3836, 3841, 3843, 3848, - 3850, 3855, 3857, 3860, 3862, 3865, 3867, 3870, 3872, 3876, - 3878, 3882, 3884, 3887, 3889, 3893, 3895, 3898, 3900, 3901, - 3903, 3907, 3909, 3913, 3917, 3919, 3923, 3927, 3928, 3930, - 3932, 3934, 3936, 3938, 3940, 3942, 3944, 3946, 3948, 3950, - 3952, 3954, 3956, 3961, 3965, 3968, 3972, 3973, 3977, 3981, - 3984, 3987, 3989, 3990, 3993, 3996, 4000, 4003, 4005, 4007, - 4011, 4017, 4019, 4022, 4027, 4030, 4031, 4033, 4034, 4036, - 4039, 4043, 4049, 4057, 4065, 4067, 4068, 4069, 4072, 4073, - 4076, 4080, 4084, 4088, 4094, 4102, 4110, 4111, 4114, 4116, - 4117, 4119, 4120, 4122, 4126, 4128, 4131, 4135, 4138, 4140, - 4145, 4148, 4150, 4151, 4155, 4157, 4161, 4163, 4166, 4171, - 4174, 4175, 4177, 4181, 4183, 4187, 4189, 4192, 4194, 4198, - 4200, 4202, 4205, 4207, 4209, 4212, 4214, 4216, 4219, 4227, - 4230, 4236, 4240, 4244, 4246, 4248, 4250, 4252, 4254, 4256, - 4258, 4260, 4262, 4264, 4266, 4268, 4270, 4272, 4275, 4278, - 4282, 4286, 4287, 4289, 4291, 4293, 4299, 4303, 4304, 4306, - 4308, 4310, 4312, 4314, 4319, 4327, 4334, 4337, 4338, 4340, - 4342, 4344, 4346, 4360, 4377, 4379, 4382, 4383, 4385, 4386, - 4388, 4389, 4392, 4393, 4395, 4396, 4403, 4412, 4419, 4428, - 4435, 4444, 4448, 4451, 4453, 4454, 4459, 4466, 4470, 4473, - 4478, 4482, 4488, 4490, 4491, 4493, 4495, 4496, 4498, 4500, - 4502, 4504, 4506, 4508, 4510, 4512, 4514, 4516, 4518, 4520, - 4522, 4524, 4526, 4528, 4530, 4532, 4534, 4536, 4538, 4540, - 4542, 4544, 4546, 4548, 4550, 4552, 4554, 4556, 4558, 4560, - 4562, 4564, 4566, 4568, 4572, 4574, 4576, 4578, 4580, 4582, - 4584, 4587, 4589, 4591, 4594, 4598, 4602, 4606, 4608, 4612, - 4616, 4619, 4623, 4627, 4629, 4631, 4633, 4637, 4643, 4645, - 4647, 4649, 4651, 4655, 4658, 4661, 4665, 4670, 4676, 4678, - 4680, 4682, 4684, 4689, 4696, 4702, 4707, 4714, 4716, 4718, - 4720, 4722, 4724, 4726, 4727, 4729, 4733, 4735, 4736, 4744, - 4748, 4750, 4753, 4757, 4760, 4761, 4764, 4765, 4768, 4773, - 4779, 4788, 4792, 4798, 4800, 4801, 4804, 4805, 4808, 4812, - 4816, 4820, 4822, 4824, 4826, 4829, 4833, 4836, 4839, 4842, - 4845, 4849, 4854, 4858, 4860, 4862, 4864, 4866, 4867, 4869, - 4873, 4875, 4879, 4882, 4892, 4905, 4917, 4930, 4945, 4949, - 4954, 4959, 4960, 4968, 4979, 4989, 4992, 4996, 4997, 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, 6176, 6178, 6180, 6182, - 6184, 6186, 6188, 6190, 6192, 6194, 6196, 6198, 6200, 6202, - 6204, 6206, 6208, 6210, 6212, 6214, 6216, 6218, 6220, 6222, - 6224, 6226, 6228, 6230, 6232 -}; - -/* YYRHS -- A `-1'-separated list of the rules' RHS. */ -static const yytype_int16 yyrhs[] = -{ - 523, 0, -1, 524, -1, 524, 514, 525, -1, 525, - -1, 910, -1, 579, -1, 526, -1, 944, -1, 945, - -1, 957, -1, 911, -1, 647, -1, 960, -1, 643, - -1, 900, -1, 573, -1, 589, -1, 569, -1, 537, - -1, 940, -1, 946, -1, 637, -1, 575, -1, 915, - -1, 913, -1, 914, -1, 903, -1, 548, -1, 932, - -1, 572, -1, 897, -1, 546, -1, 665, -1, 586, - -1, 646, -1, 588, -1, 935, -1, 949, -1, 926, - -1, 952, -1, 958, -1, -1, 32, 413, 752, 534, - -1, 32, 413, 191, 152, 752, 534, -1, 32, 202, - 538, 534, -1, 32, 202, 191, 152, 538, 534, -1, - 32, 378, 538, 534, -1, 32, 378, 191, 152, 538, - 534, -1, 32, 463, 538, 534, -1, 32, 463, 191, - 152, 538, 534, -1, 529, -1, 527, 529, -1, 383, - 117, 800, -1, 137, 117, -1, 355, -1, 355, 581, - 582, -1, 383, 583, -1, 383, 175, 636, -1, 533, - -1, 530, 515, 533, -1, 26, 616, -1, 26, 191, - 274, 152, 616, -1, 26, 82, 616, -1, 26, 82, - 191, 274, 152, 616, -1, 32, 547, 539, 528, -1, - 32, 547, 539, 137, 274, 279, -1, 32, 547, 539, - 383, 274, 279, -1, 32, 547, 539, 383, 399, 585, - -1, 32, 547, 539, 383, 604, -1, 32, 547, 539, - 353, 604, -1, 32, 547, 539, 383, 402, 539, -1, - 32, 547, 539, 26, 175, 636, 40, 190, 592, -1, - 32, 547, 539, 527, -1, 32, 547, 539, 137, 190, - -1, 32, 547, 539, 137, 190, 191, 152, -1, 137, - 547, 191, 152, 539, 641, -1, 137, 547, 539, 641, - -1, 32, 547, 539, 536, 436, 765, 761, 532, -1, - 32, 547, 539, 535, -1, 26, 606, -1, 32, 93, - 885, 590, -1, 454, 93, 885, -1, 137, 93, 191, - 152, 885, 641, -1, 137, 93, 885, 641, -1, 383, - 244, -1, 383, 445, -1, 383, 604, -1, 353, 604, - -1, 535, -1, 451, 800, -1, -1, 600, -1, 383, - 600, -1, 26, 600, -1, 137, 614, -1, 531, -1, - 534, 515, 531, -1, 293, 511, 530, 512, -1, 383, - 107, -1, 383, -1, -1, 111, 885, -1, 111, 324, - 885, -1, 111, 30, -1, 111, 324, 30, -1, 540, - -1, 539, 542, -1, 3, -1, 963, -1, 964, -1, - 539, -1, 5, -1, 5, -1, 543, -1, 542, 543, - -1, 513, 544, -1, 545, -1, 3, -1, 967, -1, - 963, -1, 969, -1, 32, 369, 885, 349, 425, 885, - -1, 32, 413, 752, 349, 425, 885, -1, 32, 413, - 191, 152, 752, 349, 425, 885, -1, 32, 378, 538, - 349, 425, 885, -1, 32, 378, 191, 152, 538, 349, - 425, 885, -1, 32, 463, 538, 349, 425, 885, -1, - 32, 463, 191, 152, 538, 349, 425, 885, -1, 32, - 202, 538, 349, 425, 885, -1, 32, 202, 191, 152, - 538, 349, 425, 885, -1, 32, 413, 752, 349, 547, - 885, 425, 885, -1, 32, 413, 191, 152, 752, 349, - 547, 885, 425, 885, -1, 32, 413, 752, 349, 93, - 885, 425, 885, -1, 32, 413, 191, 152, 752, 349, - 93, 885, 425, 885, -1, 82, -1, -1, 553, 212, - 556, 219, 550, 551, 549, 557, 559, -1, 665, -1, - 303, 560, 456, 665, -1, 511, 564, 512, 665, -1, - 511, 564, 512, 303, 560, 456, 665, -1, 117, 457, - -1, 538, -1, 538, 40, 539, -1, 59, 265, -1, - 59, 319, -1, -1, 511, 567, 512, 758, -1, 289, - 93, 885, -1, -1, 677, -1, -1, 539, 866, -1, - 568, 500, 800, -1, 511, 561, 512, 500, 800, -1, - 294, 351, -1, 294, 192, -1, -1, 289, 91, 552, - 133, 448, 383, 566, 758, -1, 289, 91, 552, 133, - 275, -1, -1, 539, 562, 563, 693, 694, -1, 808, - 562, 563, 693, 694, -1, 511, 800, 512, 562, 563, - 693, 694, -1, 357, 872, -1, -1, 450, -1, 412, - -1, 568, -1, 561, 515, 568, -1, 80, 892, -1, - -1, 892, -1, -1, 554, -1, 564, 515, 554, -1, - 555, -1, 565, 515, 555, -1, 565, -1, 565, 515, - -1, 558, -1, 567, 515, 558, -1, 539, 866, -1, - 100, 436, 538, 40, 144, 666, -1, 100, 436, 538, - 40, 144, 511, 570, 512, -1, 100, 436, 538, 40, - 765, -1, 571, -1, -1, 541, -1, 571, 515, 541, - -1, 321, 539, -1, 321, 539, 500, 931, -1, 321, - 539, 511, 846, 512, -1, 100, 635, 378, 538, 574, - -1, 100, 635, 378, 191, 274, 152, 538, 574, -1, - 100, 294, 351, 635, 378, 538, 574, -1, 580, -1, - -1, 151, 885, 578, -1, 100, 635, 413, 962, 40, - 151, 885, 578, 961, -1, 100, 635, 413, 191, 274, - 152, 962, 40, 151, 885, 578, 961, -1, 800, -1, - 895, 13, 800, -1, 576, -1, 577, 515, 576, -1, - 511, 577, 512, -1, -1, 32, 378, 538, 580, -1, - 32, 378, 191, 152, 538, 580, -1, 583, -1, 580, - 583, -1, 473, -1, 497, -1, -1, 4, -1, 502, - 4, -1, 503, 4, -1, 585, -1, 40, 768, -1, - 60, 582, -1, 106, -1, 272, 106, -1, 201, 584, - 582, -1, 250, 582, -1, 260, 582, -1, 272, 250, - -1, 272, 260, -1, 304, 59, 892, -1, 378, 265, - 892, -1, 397, 581, 582, -1, 355, -1, 355, 581, - 582, -1, 59, -1, -1, 888, -1, 502, 888, -1, - 503, 888, -1, 22, 587, -1, 52, 587, -1, 397, - 587, -1, 86, 587, -1, 143, 587, -1, 362, 587, - -1, 476, -1, 427, -1, -1, 449, 538, -1, 100, - 635, 413, 538, 511, 622, 512, 611, 603, -1, 100, - 635, 413, 191, 274, 152, 538, 511, 622, 512, 611, - 603, -1, 100, 294, 351, 635, 413, 538, 511, 622, - 512, 611, 603, -1, -1, 590, 615, -1, 630, -1, - 969, -1, 838, -1, 582, -1, 541, -1, 273, -1, - 511, 580, 512, -1, -1, 541, -1, 272, 25, -1, - 356, -1, 63, -1, 383, 279, -1, 383, 117, -1, - 93, 885, 596, -1, 596, -1, 610, -1, 80, 892, - -1, 274, 279, -1, 279, -1, 442, 621, -1, 327, - 226, 621, -1, 74, 511, 800, 512, 605, -1, 451, - 88, 885, -1, 117, 801, -1, 343, 538, 624, 633, - 602, -1, 465, -1, 403, -1, 597, -1, -1, 175, - 636, 40, 190, 592, -1, 175, 636, 40, 511, 800, - 512, 598, -1, 40, 511, 800, 512, 598, -1, 614, - 593, -1, 289, 448, 594, -1, 601, -1, 626, -1, - 601, 626, -1, 626, 601, -1, -1, 289, 86, 137, - -1, 289, 86, 122, 365, -1, 289, 86, 326, 365, - -1, -1, 511, 608, 512, -1, 272, 204, -1, -1, - 93, 885, 631, -1, 631, -1, 85, -1, 94, -1, - 118, -1, 190, -1, 203, -1, 399, -1, 402, -1, - 30, -1, 627, -1, 608, 515, 627, -1, 451, 202, - 618, -1, 119, -1, 274, 119, -1, 206, 120, -1, - 206, 194, -1, 473, 604, -1, 473, 287, -1, 475, - 287, -1, -1, 511, 617, 512, -1, 613, 200, 607, - -1, 613, 149, 607, -1, -1, 545, -1, 274, 119, - -1, 119, -1, 206, 194, -1, 206, 120, -1, 274, - 453, -1, 272, 204, -1, 539, 765, 625, -1, 539, - 764, 599, 625, -1, 620, -1, 617, 515, 620, -1, - 539, -1, 616, -1, 634, -1, 606, -1, 545, 500, - 591, -1, 545, -1, 473, 612, -1, -1, 632, -1, - 632, 515, -1, -1, 539, -1, 511, 628, 512, -1, - -1, 625, 595, -1, -1, 289, 122, 594, -1, 545, - 500, 591, -1, 545, -1, 545, 513, 545, 500, 591, - -1, 545, 513, 545, -1, 623, -1, 628, 515, 623, - -1, 628, -1, 628, 515, -1, 765, -1, 889, 893, - 506, 436, -1, 384, 889, 893, 506, 436, -1, 74, - 511, 800, 512, 590, -1, 442, 511, 629, 512, 621, - 590, -1, 442, 609, 590, -1, 327, 226, 511, 629, - 512, 621, 590, -1, 327, 226, 609, 590, -1, 168, - 226, 511, 629, 512, 343, 538, 624, 633, 602, 590, - -1, 619, -1, 632, 515, 619, -1, 248, 172, -1, - 248, 308, -1, 248, 389, -1, -1, 236, 538, 613, - -1, 419, -1, 417, -1, 240, 419, -1, 240, 417, - -1, 177, 419, -1, 177, 417, -1, 445, -1, -1, - 33, -1, 59, 117, -1, 137, 638, 191, 152, 640, - 641, -1, 137, 638, 640, 641, -1, 137, 639, 191, - 152, 882, 641, -1, 137, 639, 882, 641, -1, 137, - 642, 885, 289, 892, 641, -1, 137, 642, 191, 152, - 885, 289, 892, 641, -1, 413, -1, 378, -1, 173, - -1, 245, -1, 245, 413, -1, 463, -1, 249, 463, - -1, 202, -1, 168, 413, -1, 81, -1, 97, -1, - 369, -1, 399, -1, 420, 372, 307, -1, 420, 372, - 129, -1, 420, 372, 418, -1, 420, 372, 90, -1, - 436, -1, 24, 251, -1, 146, 430, -1, 156, -1, - 168, 107, 477, -1, 333, -1, 381, -1, 892, -1, - 640, 515, 892, -1, 63, -1, 356, -1, -1, 318, - -1, 366, -1, 430, -1, 100, 635, 644, 538, 645, - 40, 413, 665, -1, 100, 635, 644, 191, 274, 152, - 538, 645, 40, 413, 665, -1, 100, 294, 351, 635, - 644, 538, 645, 40, 413, 665, -1, 100, 635, 644, - 538, 645, 40, 800, -1, 100, 635, 644, 191, 274, - 152, 538, 645, 40, 800, -1, 100, 294, 351, 635, - 644, 538, 645, 40, 800, -1, 173, -1, 245, -1, - 511, 512, -1, 511, 846, 512, -1, 553, 448, 941, - 383, 566, 732, 942, 559, -1, 98, 660, 538, 624, - 658, 649, 654, 663, 650, 581, 655, -1, 98, 511, - 665, 512, 425, 654, 663, 581, 655, -1, 98, 171, - 108, 539, 425, 539, 648, -1, -1, 511, 369, 512, - -1, 511, 107, 512, -1, 171, -1, 425, -1, 652, - 124, 541, -1, -1, 662, -1, 651, 515, 662, -1, - 451, -1, -1, 40, -1, -1, 332, -1, -1, 659, - -1, 511, 664, 512, -1, 923, -1, 582, -1, 504, - -1, 511, 651, 512, -1, 805, -1, -1, 545, 656, - -1, 473, 287, -1, -1, 659, 661, -1, -1, 55, - -1, -1, 55, -1, 287, -1, 170, -1, 123, 653, - 541, -1, 279, 653, 541, -1, 102, -1, 186, -1, - 335, 653, 541, -1, 145, 653, 541, -1, 167, 335, - 628, -1, 167, 335, 504, -1, 309, 59, 628, -1, - 309, 59, 504, -1, 167, 274, 279, 628, -1, 167, - 279, 628, -1, 141, 541, -1, 923, -1, 541, -1, - 400, -1, 401, -1, 657, -1, 664, 515, 657, -1, - 667, -1, 666, -1, 511, 667, 512, -1, 511, 666, - 512, -1, 670, -1, 668, 690, -1, 668, 689, 723, - 696, -1, 668, 689, 695, 724, -1, 677, 668, -1, - 677, 668, 690, -1, 677, 668, 689, 723, 696, -1, - 677, 668, 689, 695, 724, -1, 670, -1, 666, -1, - 376, 687, 871, -1, -1, 376, 687, 871, 681, 732, - 758, 712, 721, 814, 722, 700, -1, 376, 686, 873, - 681, 732, 758, 712, 721, 814, 722, 700, -1, 171, - 733, 669, 681, 758, 712, 721, 814, 722, 700, -1, - 171, 733, 376, 686, 873, 681, 758, 712, 721, 814, - 722, 700, -1, 731, -1, 413, 752, -1, 668, 441, - 684, 685, 668, -1, 668, 441, 684, 668, -1, 668, - 217, 684, 668, -1, 668, 147, 684, 668, -1, 672, - 735, 451, 873, -1, 672, 735, 451, 873, 180, 59, - 884, -1, 672, 735, 180, 59, 884, -1, 672, 735, - 289, 676, -1, 672, 735, 289, 676, 180, 59, 884, - -1, 672, 735, 289, 676, 451, 873, -1, 672, 735, - 289, 676, 451, 873, 180, 59, 884, -1, 673, 735, - 289, 873, 219, 265, 885, 671, 884, -1, 673, 735, - 289, 873, -1, 456, -1, 457, -1, 313, -1, 315, - -1, 446, -1, 314, -1, 801, -1, 801, 198, 511, - 667, 512, -1, 738, -1, 674, -1, 675, 515, 674, - -1, 675, -1, 675, 515, -1, 473, 678, -1, 497, - 678, -1, 473, 341, 678, -1, 679, -1, 678, 515, - 679, -1, 885, 894, 40, 680, 511, 899, 512, -1, - 249, -1, 274, 249, -1, -1, 219, 682, -1, -1, - 419, 683, 538, -1, 417, 683, 538, -1, 240, 419, - 683, 538, -1, 240, 417, 683, 538, -1, 177, 419, - 683, 538, -1, 177, 417, 683, 538, -1, 445, 683, - 538, -1, 413, 538, -1, 538, -1, 413, -1, -1, - 30, -1, 132, -1, -1, 59, 265, -1, 132, -1, - 132, 289, 511, 844, 512, -1, 30, -1, -1, 192, - 281, -1, 354, 281, -1, -1, 690, -1, -1, 295, - 59, 691, -1, 295, 59, 30, 693, 694, -1, 692, - -1, 691, 515, 692, -1, 800, 451, 838, 694, -1, - 800, 693, 694, -1, 41, -1, 126, -1, -1, 496, - 163, -1, 496, 230, -1, -1, 697, 698, -1, 698, - 697, -1, 697, -1, 698, -1, 695, -1, -1, 237, - 706, -1, 237, 706, 515, 707, -1, 161, 711, 708, - 710, 290, -1, 161, 711, 710, 290, -1, 286, 707, - -1, 286, 708, 710, -1, 4, 506, -1, 9, 506, - -1, 4, 312, -1, 9, 312, -1, 9, -1, 9, - 365, -1, 451, 367, 702, -1, -1, 539, -1, -1, - 701, 511, 699, 512, 705, -1, 699, -1, 699, 511, - 539, 512, -1, 699, 511, 539, 515, 9, 512, -1, - 415, 702, -1, 703, -1, -1, 350, 511, 9, 512, - -1, -1, 800, -1, 30, -1, 800, 506, -1, 4, - 312, -1, 9, 312, -1, 800, -1, 802, -1, 502, - 709, -1, 503, 709, -1, 888, -1, 4, -1, 364, - -1, 365, -1, 163, -1, 271, -1, 180, 59, 714, - -1, 180, 59, 30, -1, -1, 715, -1, 713, 515, - 715, -1, 713, -1, 713, 515, -1, 800, -1, 716, - -1, 718, -1, 717, -1, 719, -1, 511, 512, -1, - 363, 511, 844, 512, -1, 103, 511, 844, 512, -1, - 181, 385, 511, 714, 512, -1, 181, -1, 182, -1, - 185, 800, -1, -1, 334, 800, -1, -1, 725, -1, - 166, 337, 290, -1, 723, -1, -1, 726, -1, 725, - 726, -1, 727, 728, 729, -1, 166, 448, -1, 166, - 272, 226, 448, -1, 166, 386, -1, 166, 226, 386, - -1, 284, 881, -1, -1, 278, -1, 390, 243, -1, - -1, 457, 511, 844, 512, -1, 730, 515, 511, 844, - 512, -1, 730, -1, 730, 515, -1, 171, 734, -1, - -1, 735, -1, 733, 515, 735, -1, 733, -1, 733, - 515, -1, 752, 747, 704, -1, 753, 748, 704, -1, - 731, 746, 704, -1, 231, 753, 748, -1, 666, 747, - 704, -1, 231, 666, 747, -1, 745, -1, 511, 745, - 512, 746, -1, 735, 313, 511, 873, 166, 741, 736, - 512, 747, -1, 735, 446, 737, 511, 742, 166, 744, - 512, 747, -1, 180, 59, 883, -1, -1, 199, 281, - -1, 148, 281, -1, -1, 801, 198, 511, 873, 512, - -1, 801, 198, 540, -1, 803, -1, 511, 842, 512, - -1, 739, 198, 511, 873, 512, -1, 739, 198, 540, - -1, 740, -1, 741, 740, -1, 540, -1, 511, 883, - 512, -1, 742, 198, 511, 873, 512, -1, 743, -1, - 744, 743, -1, 511, 745, 512, -1, 735, 101, 224, - 735, -1, 735, 749, 224, 735, 751, -1, 735, 224, - 735, 751, -1, 735, 268, 749, 224, 735, -1, 735, - 268, 224, 735, -1, 735, 42, 749, 224, 735, 751, - -1, 735, 42, 224, 735, 751, -1, 735, 320, 224, - 735, -1, 735, 37, 224, 735, 751, -1, 735, 377, - 224, 735, 751, -1, 40, 540, 511, 883, 512, -1, - 40, 540, -1, 539, 511, 883, 512, -1, 539, -1, - 746, -1, -1, 746, -1, 40, 511, 759, 512, -1, - 40, 540, 511, 759, 512, -1, 539, 511, 759, 512, - -1, -1, 172, 750, -1, 234, 750, -1, 360, 750, - -1, 377, -1, 37, -1, 208, -1, 299, -1, -1, - 451, 511, 883, 512, -1, 289, 800, -1, 538, -1, - 538, 504, -1, 290, 538, -1, 290, 511, 538, 512, - -1, 808, 757, -1, 365, 171, 511, 755, 512, 757, - -1, 808, 756, -1, 754, -1, 755, 515, 754, -1, - 40, 511, 759, 512, -1, -1, 497, 296, -1, -1, - 470, 800, -1, -1, 760, -1, 759, 515, 760, -1, - 540, 765, 761, -1, 80, 892, -1, -1, 539, 765, - -1, 762, 515, 539, 765, -1, 364, -1, 406, -1, - 765, -1, -1, 768, 767, -1, 384, 768, 767, -1, - 768, 39, 509, 888, 510, -1, 384, 768, 39, 509, - 888, 510, -1, 768, 39, -1, 384, 768, 39, -1, - 766, -1, 763, 511, 762, 512, 767, -1, 246, 511, - 848, 512, 767, -1, 441, 511, 762, 512, 767, -1, - 3, 513, 3, -1, 766, 513, 3, -1, 767, 509, - 510, -1, 767, 509, 888, 510, -1, -1, 770, -1, - 772, -1, 774, -1, 778, -1, 784, -1, 785, 799, - -1, 785, 511, 888, 512, -1, 772, -1, 775, -1, - 779, -1, 784, -1, 891, 771, -1, 511, 845, 512, - -1, -1, 215, -1, 216, -1, 391, -1, 54, -1, - 338, -1, 164, 773, -1, 136, 323, -1, 115, 771, - -1, 112, 771, -1, 282, 771, -1, 57, -1, 511, - 888, 512, -1, -1, 776, -1, 777, -1, 776, -1, - 777, -1, 56, 783, 511, 844, 512, -1, 56, 783, - -1, 780, -1, 781, -1, 780, -1, 781, -1, 782, - 511, 888, 512, -1, 782, -1, 72, 783, -1, 71, - 783, -1, 458, -1, 267, 72, 783, -1, 267, 71, - 783, -1, 269, 783, -1, 460, -1, -1, 424, 511, - 888, 512, 786, -1, 424, 786, -1, 423, 511, 888, - 512, 786, -1, 423, 786, -1, 218, -1, 497, 423, - 494, -1, 475, 423, 494, -1, -1, 491, -1, 492, - -1, 262, -1, 263, -1, 109, -1, 110, -1, 188, - -1, 189, -1, 258, -1, 259, -1, 373, -1, 374, - -1, 256, -1, 257, -1, 252, -1, 253, -1, 467, - -1, 468, -1, 113, -1, 114, -1, 69, -1, 68, - -1, 255, -1, 254, -1, 787, -1, 788, -1, 789, - -1, 790, -1, 791, -1, 792, -1, 793, -1, 794, - -1, 795, -1, 796, -1, 797, -1, 798, -1, 787, - 425, 788, -1, 789, 425, 790, -1, 789, 425, 791, - -1, 789, 425, 792, -1, 790, 425, 791, -1, 790, - 425, 792, -1, 791, 425, 792, -1, -1, 802, -1, - 800, 11, 765, -1, 800, 80, 892, -1, 800, 46, - 423, 494, 800, -1, 502, 800, -1, 503, 800, -1, - 800, 502, 800, -1, 800, 503, 800, -1, 800, 504, - 800, -1, 800, 505, 800, -1, 800, 15, 800, -1, - 800, 506, 800, -1, 800, 507, 800, -1, 800, 16, - 800, -1, 800, 498, 800, -1, 800, 499, 800, -1, - 800, 500, 800, -1, 800, 19, 800, -1, 800, 20, - 800, -1, 800, 21, 800, -1, 800, 837, 800, -1, - 837, 800, -1, 800, 837, -1, 800, 36, 800, -1, - 800, 294, 800, -1, 274, 800, -1, 495, 800, -1, - 800, 176, 800, -1, 800, 236, 800, -1, 800, 236, - 800, 145, 800, -1, 800, 495, 236, 800, -1, 800, - 495, 236, 800, 145, 800, -1, 800, 193, 800, -1, - 800, 193, 800, 145, 800, -1, 800, 495, 193, 800, - -1, 800, 495, 193, 800, 145, 800, -1, 800, 388, - 425, 800, -1, 800, 388, 425, 800, 145, 800, -1, - 800, 495, 388, 425, 800, -1, 800, 495, 388, 425, - 800, 145, 800, -1, 800, 221, 279, -1, 800, 222, - -1, 800, 221, 274, 279, -1, 800, 274, 279, -1, - 800, 277, -1, 800, 17, 800, -1, 800, 18, 800, - -1, 826, 301, 826, -1, 800, 221, 432, -1, 800, - 221, 274, 432, -1, 800, 221, 159, -1, 800, 221, - 274, 159, -1, 800, 221, 443, -1, 800, 221, 274, - 443, -1, 800, 221, 132, 171, 800, -1, 800, 221, - 274, 132, 171, 800, -1, 800, 221, 284, 511, 848, - 512, -1, 800, 221, 274, 284, 511, 848, 512, -1, - 800, 53, 870, 801, 36, 800, -1, 800, 495, 53, - 870, 801, 36, 800, -1, 800, 53, 410, 801, 36, - 800, -1, 800, 495, 53, 410, 801, 36, 800, -1, - 800, 198, 858, -1, 800, 495, 198, 858, -1, 800, - 839, 834, 666, -1, 800, 839, 834, 511, 800, 512, - -1, 117, -1, 83, 511, 800, 512, -1, 504, 876, - 880, -1, 539, 513, 504, 876, 880, -1, 802, -1, - 801, 11, 765, -1, 502, 801, -1, 503, 801, -1, - 801, 502, 801, -1, 801, 503, 801, -1, 801, 504, - 801, -1, 801, 505, 801, -1, 801, 15, 801, -1, - 801, 506, 801, -1, 801, 507, 801, -1, 801, 16, - 801, -1, 801, 498, 801, -1, 801, 499, 801, -1, - 801, 500, 801, -1, 801, 19, 801, -1, 801, 20, - 801, -1, 801, 21, 801, -1, 801, 837, 801, -1, - 837, 801, -1, 801, 837, -1, 801, 221, 132, 171, - 801, -1, 801, 221, 274, 132, 171, 801, -1, 801, - 221, 284, 511, 848, 512, -1, 801, 221, 274, 284, - 511, 848, 512, -1, 803, -1, 826, -1, 804, 869, - -1, 864, -1, 887, -1, 516, 9, -1, 517, 545, - -1, 509, 845, 510, -1, 810, -1, 39, 666, -1, - 39, 509, 845, 510, -1, 859, -1, 666, -1, 666, - 542, -1, 152, 666, -1, 720, 511, 844, 512, -1, - 518, -1, 10, -1, 511, 800, 512, -1, 805, -1, - 246, 519, 833, 520, -1, 807, -1, 519, 829, 520, - -1, 886, 511, 512, -1, 886, 511, 846, 689, 688, - 512, -1, 886, 511, 459, 847, 689, 688, 512, -1, - 886, 511, 846, 515, 459, 847, 689, 688, 512, -1, - 886, 511, 30, 846, 689, 688, 512, -1, 886, 511, - 132, 846, 689, 688, 512, -1, 806, 811, 812, 813, - 817, -1, 809, -1, 806, -1, 809, -1, 81, 166, - 511, 800, 512, -1, 66, 511, 800, 40, 765, 512, - -1, 435, 511, 800, 40, 765, 512, -1, 158, 511, - 849, 512, -1, 302, 511, 851, 512, -1, 319, 511, - 853, 512, -1, 408, 511, 854, 512, -1, 429, 511, - 800, 40, 765, 512, -1, 431, 511, 58, 857, 512, - -1, 431, 511, 232, 857, 512, -1, 431, 511, 426, - 857, 512, -1, 431, 511, 857, 512, -1, 280, 511, - 800, 515, 800, 512, -1, 79, 511, 844, 512, -1, - 509, 800, 166, 539, 198, 800, 510, -1, 509, 800, - 166, 539, 198, 802, 191, 800, 510, -1, 474, 180, - 511, 690, 512, -1, -1, 162, 511, 470, 800, 512, - -1, 162, 511, 800, 512, -1, -1, 155, -1, -1, - 472, 815, -1, -1, 816, -1, 815, 515, 816, -1, - 539, 40, 818, -1, 300, 818, -1, 300, 539, -1, - -1, 511, 819, 820, 689, 821, 512, -1, 539, -1, - -1, 309, 59, 843, -1, -1, 336, 822, 824, -1, - 365, 822, 824, -1, 183, 822, 824, -1, -1, 823, - -1, 53, 823, 36, 823, -1, 438, 322, -1, 438, - 165, -1, 104, 364, -1, 800, 322, -1, 800, 165, - -1, 148, 104, 364, -1, 148, 180, -1, 148, 422, - -1, 148, 272, 297, -1, -1, 364, 511, 844, 512, - -1, 364, 511, 512, -1, 825, -1, 511, 843, 515, - 800, 512, -1, 540, 521, 800, -1, 827, -1, 828, - 515, 827, -1, 828, -1, 828, 515, -1, 800, 521, - 800, -1, 830, -1, 831, 515, 830, -1, 831, -1, - 831, 515, -1, 832, -1, -1, 38, -1, 393, -1, - 30, -1, 8, -1, 836, -1, 502, -1, 503, -1, - 504, -1, 505, -1, 15, -1, 506, -1, 507, -1, - 16, -1, 498, -1, 499, -1, 500, -1, 19, -1, - 20, -1, 21, -1, 8, -1, 291, 511, 840, 512, - -1, 835, -1, 291, 511, 840, 512, -1, 835, -1, - 291, 511, 840, 512, -1, 236, -1, 495, 236, -1, - 176, -1, 495, 176, -1, 193, -1, 495, 193, -1, - 835, -1, 539, 513, 840, -1, 802, -1, 841, 515, - 802, -1, 841, -1, 841, 515, -1, 800, -1, 843, - 515, 800, -1, 843, -1, 843, 515, -1, 844, -1, - -1, 847, -1, 846, 515, 847, -1, 800, -1, 895, - 13, 800, -1, 895, 14, 800, -1, 765, -1, 848, - 515, 765, -1, 850, 171, 800, -1, -1, 3, -1, - 787, -1, 788, -1, 789, -1, 790, -1, 791, -1, - 792, -1, 793, -1, 794, -1, 795, -1, 796, -1, - 797, -1, 798, -1, 541, -1, 800, 852, 855, 856, - -1, 800, 852, 855, -1, 316, 800, -1, 801, 198, - 801, -1, -1, 800, 855, 856, -1, 800, 856, 855, - -1, 800, 855, -1, 800, 856, -1, 843, -1, -1, - 171, 800, -1, 166, 800, -1, 800, 171, 844, -1, - 171, 844, -1, 844, -1, 666, -1, 511, 844, 512, - -1, 65, 863, 860, 862, 143, -1, 861, -1, 860, - 861, -1, 469, 800, 421, 800, -1, 139, 800, -1, - -1, 800, -1, -1, 539, -1, 539, 542, -1, 509, - 800, 510, -1, 509, 865, 521, 865, 510, -1, 509, - 865, 521, 865, 521, 865, 510, -1, 509, 865, 521, - 503, 521, 865, 510, -1, 800, -1, -1, -1, 866, - 543, -1, -1, 511, 512, -1, 511, 846, 512, -1, - 513, 544, 867, -1, 509, 800, 510, -1, 509, 865, - 521, 865, 510, -1, 509, 865, 521, 865, 521, 865, - 510, -1, 509, 865, 521, 503, 521, 865, 510, -1, - -1, 869, 868, -1, 45, -1, -1, 873, -1, -1, - 874, -1, 872, 515, 874, -1, 872, -1, 872, 515, - -1, 800, 40, 896, -1, 800, 3, -1, 800, -1, - 148, 511, 883, 512, -1, 148, 539, -1, 875, -1, - -1, 800, 40, 539, -1, 877, -1, 878, 515, 877, - -1, 878, -1, 878, 515, -1, 351, 511, 879, 512, - -1, 351, 877, -1, -1, 538, -1, 881, 515, 538, - -1, 885, -1, 882, 515, 885, -1, 882, -1, 882, - 515, -1, 883, -1, 511, 883, 512, -1, 540, -1, - 890, -1, 539, 542, -1, 888, -1, 4, -1, 541, - 866, -1, 6, -1, 7, -1, 886, 541, -1, 886, - 511, 846, 689, 688, 512, 541, -1, 769, 541, -1, - 785, 511, 800, 512, 799, -1, 785, 888, 799, -1, - 785, 541, 799, -1, 432, -1, 159, -1, 279, -1, - 9, -1, 3, -1, 963, -1, 968, -1, 3, -1, - 963, -1, 965, -1, 3, -1, 963, -1, 966, -1, - 539, -1, 539, 893, -1, 513, 544, -1, 893, 513, - 544, -1, 511, 883, 512, -1, -1, 889, -1, 545, - -1, 5, -1, 324, 885, 898, 40, 899, -1, 511, - 848, 512, -1, -1, 665, -1, 548, -1, 646, -1, - 647, -1, 940, -1, 100, 369, 538, 901, -1, 100, - 369, 191, 274, 152, 538, 901, -1, 100, 294, 351, - 369, 538, 901, -1, 901, 902, -1, -1, 589, -1, - 903, -1, 573, -1, 958, -1, 100, 909, 202, 906, - 907, 289, 538, 905, 511, 567, 512, 908, 758, -1, - 100, 909, 202, 906, 191, 274, 152, 618, 289, 538, - 905, 511, 567, 512, 908, 758, -1, 539, -1, 451, - 904, -1, -1, 89, -1, -1, 618, -1, -1, 473, - 604, -1, -1, 442, -1, -1, 32, 413, 752, 383, - 369, 885, -1, 32, 413, 191, 152, 752, 383, 369, - 885, -1, 32, 378, 538, 383, 369, 885, -1, 32, - 378, 191, 152, 538, 383, 369, 885, -1, 32, 463, - 538, 383, 369, 885, -1, 32, 463, 191, 152, 538, - 383, 369, 885, -1, 167, 75, 912, -1, 75, 912, - -1, 539, -1, -1, 154, 108, 541, 655, -1, 154, - 108, 539, 425, 541, 655, -1, 197, 108, 541, -1, - 153, 918, -1, 153, 922, 916, 918, -1, 153, 461, - 918, -1, 153, 511, 921, 512, 918, -1, 461, -1, - -1, 923, -1, 582, -1, -1, 910, -1, 579, -1, - 526, -1, 957, -1, 911, -1, 647, -1, 960, -1, - 643, -1, 900, -1, 573, -1, 589, -1, 569, -1, - 537, -1, 940, -1, 637, -1, 575, -1, 903, -1, - 548, -1, 932, -1, 572, -1, 897, -1, 546, -1, - 665, -1, 586, -1, 646, -1, 935, -1, 949, -1, - 926, -1, 952, -1, 958, -1, 3, -1, 963, -1, - 967, -1, 919, -1, 541, -1, 924, -1, 921, 515, - 924, -1, 35, -1, 34, -1, 432, -1, 159, -1, - 289, -1, 920, -1, 925, 917, -1, 919, -1, 922, - -1, 383, 927, -1, 383, 240, 927, -1, 383, 382, - 927, -1, 383, 177, 927, -1, 928, -1, 955, 171, - 104, -1, 423, 494, 930, -1, 369, 541, -1, 955, - 425, 931, -1, 955, 500, 931, -1, 800, -1, 541, - -1, 3, -1, 785, 541, 799, -1, 785, 511, 888, - 512, 541, -1, 582, -1, 117, -1, 240, -1, 929, - -1, 931, 515, 929, -1, 239, 933, -1, 213, 933, - -1, 167, 213, 933, -1, 213, 933, 171, 934, -1, - 167, 213, 933, 171, 934, -1, 541, -1, 539, -1, - 541, -1, 539, -1, 452, 937, 939, 916, -1, 452, - 937, 939, 916, 538, 894, -1, 452, 937, 939, 916, - 944, -1, 452, 511, 938, 512, -1, 452, 511, 938, - 512, 538, 894, -1, 922, -1, 461, -1, 170, -1, - 172, -1, 3, -1, 172, -1, -1, 936, -1, 938, - 515, 936, -1, 170, -1, -1, 553, 122, 171, 941, - 943, 942, 559, -1, 433, 683, 941, -1, 752, -1, - 752, 539, -1, 752, 40, 539, -1, 470, 800, -1, - -1, 451, 734, -1, -1, 922, 916, -1, 922, 916, - 538, 894, -1, 47, 947, 541, 948, 655, -1, 47, - 191, 274, 152, 947, 541, 948, 655, -1, 128, 947, - 3, -1, 128, 108, 191, 152, 3, -1, 108, -1, - -1, 40, 539, -1, -1, 353, 951, -1, 353, 240, - 951, -1, 353, 382, 951, -1, 353, 177, 951, -1, - 955, -1, 30, -1, 950, -1, 423, 494, -1, 427, - 223, 235, -1, 953, 665, -1, 409, 665, -1, 409, - 956, -1, 953, 956, -1, 953, 423, 494, -1, 953, - 427, 223, 235, -1, 953, 30, 954, -1, 953, -1, - 387, -1, 127, -1, 414, -1, -1, 539, -1, 955, - 513, 539, -1, 539, -1, 956, 513, 539, -1, 61, - 806, -1, 100, 635, 463, 538, 624, 908, 40, 665, - 959, -1, 100, 635, 463, 191, 274, 152, 538, 624, - 908, 40, 665, 959, -1, 100, 294, 351, 635, 463, - 538, 624, 908, 40, 665, 959, -1, 100, 635, 341, - 463, 538, 511, 628, 512, 908, 40, 665, 959, -1, - 100, 294, 351, 635, 341, 463, 538, 511, 628, 512, - 908, 40, 665, 959, -1, 473, 74, 292, -1, 473, - 64, 74, 292, -1, 473, 240, 74, 292, -1, -1, - 100, 635, 413, 962, 40, 665, 961, -1, 100, 635, - 413, 191, 274, 152, 962, 40, 665, 961, -1, 100, - 294, 351, 635, 413, 962, 40, 665, 961, -1, 473, - 107, -1, 473, 272, 107, -1, -1, 538, 624, 611, - 603, -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, 69, -1, 70, -1, - 73, -1, 75, -1, 76, -1, 77, -1, 78, -1, - 84, -1, 85, -1, 86, -1, 87, -1, 88, -1, - 90, -1, 91, -1, 92, -1, 94, -1, 95, -1, - 96, -1, 97, -1, 98, -1, 99, -1, 102, -1, - 103, -1, 104, -1, 105, -1, 106, -1, 107, -1, - 108, -1, 109, -1, 110, -1, 111, -1, 113, -1, - 114, -1, 116, -1, 118, -1, 120, -1, 121, -1, - 122, -1, 123, -1, 124, -1, 125, -1, 127, -1, - 128, -1, 129, -1, 130, -1, 131, -1, 134, -1, - 135, -1, 136, -1, 137, -1, 138, -1, 140, -1, - 141, -1, 142, -1, 144, -1, 145, -1, 146, -1, - 148, -1, 149, -1, 150, -1, 151, -1, 153, -1, - 154, -1, 155, -1, 156, -1, 157, -1, 160, -1, - 162, -1, 163, -1, 165, -1, 167, -1, 169, -1, - 173, -1, 174, -1, 177, -1, 179, -1, 183, -1, - 184, -1, 186, -1, 187, -1, 188, -1, 189, -1, - 190, -1, 191, -1, 192, -1, 194, -1, 195, -1, - 196, -1, 197, -1, 199, -1, 200, -1, 201, -1, - 202, -1, 203, -1, 204, -1, 205, -1, 207, -1, - 210, -1, 211, -1, 212, -1, 213, -1, 214, -1, - 220, -1, 223, -1, 225, -1, 226, -1, 227, -1, - 228, -1, 229, -1, 230, -1, 233, -1, 235, -1, - 238, -1, 239, -1, 240, -1, 241, -1, 242, -1, - 243, -1, 244, -1, 245, -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, 261, -1, 262, -1, 263, -1, - 264, -1, 265, -1, 266, -1, 270, -1, 271, -1, - 272, -1, 275, -1, 276, -1, 278, -1, 281, -1, - 283, -1, 284, -1, 285, -1, 287, -1, 288, -1, - 291, -1, 292, -1, 293, -1, 296, -1, 297, -1, - 300, -1, 303, -1, 304, -1, 305, -1, 306, -1, - 307, -1, 308, -1, 309, -1, 310, -1, 311, -1, - 312, -1, 317, -1, 318, -1, 321, -1, 322, -1, - 324, -1, 325, -1, 326, -1, 328, -1, 329, -1, - 330, -1, 331, -1, 332, -1, 333, -1, 335, -1, - 336, -1, 337, -1, 339, -1, 340, -1, 341, -1, - 342, -1, 344, -1, 345, -1, 346, -1, 347, -1, - 348, -1, 349, -1, 350, -1, 351, -1, 352, -1, - 353, -1, 354, -1, 355, -1, 356, -1, 358, -1, - 359, -1, 361, -1, 362, -1, 363, -1, 365, -1, - 366, -1, 367, -1, 368, -1, 369, -1, 370, -1, - 371, -1, 372, -1, 373, -1, 374, -1, 375, -1, - 378, -1, 379, -1, 380, -1, 381, -1, 382, -1, - 383, -1, 385, -1, 386, -1, 387, -1, 389, -1, - 390, -1, 392, -1, 394, -1, 395, -1, 396, -1, - 397, -1, 398, -1, 399, -1, 400, -1, 401, -1, - 402, -1, 403, -1, 404, -1, 405, -1, 407, -1, - 409, -1, 411, -1, 412, -1, 414, -1, 416, -1, - 417, -1, 418, -1, 419, -1, 420, -1, 422, -1, - 427, -1, 428, -1, 430, -1, 433, -1, 434, -1, - 436, -1, 437, -1, 438, -1, 439, -1, 440, -1, - 443, -1, 444, -1, 445, -1, 447, -1, 448, -1, - 449, -1, 450, -1, 452, -1, 453, -1, 454, -1, - 455, -1, 456, -1, 460, -1, 462, -1, 463, -1, - 464, -1, 465, -1, 466, -1, 467, -1, 468, -1, - 471, -1, 474, -1, 475, -1, 476, -1, 477, -1, - 478, -1, 479, -1, 491, -1, 492, -1, 493, -1, - 494, -1, 53, -1, 54, -1, 56, -1, 57, -1, - 71, -1, 72, -1, 79, -1, 83, -1, 112, -1, - 115, -1, 152, -1, 158, -1, 164, -1, 175, -1, - 181, -1, 182, -1, 209, -1, 215, -1, 216, -1, - 218, -1, 246, -1, 267, -1, 269, -1, 273, -1, - 280, -1, 282, -1, 298, -1, 302, -1, 319, -1, - 323, -1, 338, -1, 364, -1, 384, -1, 391, -1, - 406, -1, 408, -1, 423, -1, 424, -1, 429, -1, - 431, -1, 435, -1, 457, -1, 458, -1, 480, -1, - 481, -1, 482, -1, 483, -1, 484, -1, 485, -1, - 486, -1, 487, -1, 488, -1, 489, -1, 490, -1, - 42, -1, 49, -1, 55, -1, 81, -1, 89, -1, - 101, -1, 170, -1, 172, -1, 175, -1, 176, -1, - 193, -1, 208, -1, 221, -1, 222, -1, 224, -1, - 234, -1, 236, -1, 246, -1, 268, -1, 277, -1, - 299, -1, 301, -1, 320, -1, 360, -1, 388, -1, - 406, -1, 415, -1, 461, -1, 37, -1, 42, -1, - 49, -1, 55, -1, 81, -1, 83, -1, 89, -1, - 101, -1, 170, -1, 172, -1, 176, -1, 193, -1, - 208, -1, 221, -1, 222, -1, 224, -1, 234, -1, - 236, -1, 268, -1, 277, -1, 299, -1, 301, -1, - 320, -1, 360, -1, 377, -1, 388, -1, 415, -1, - 435, -1, 461, -1, 37, -1, 42, -1, 49, -1, - 53, -1, 54, -1, 55, -1, 56, -1, 57, -1, - 72, -1, 71, -1, 79, -1, 81, -1, 83, -1, - 89, -1, 101, -1, 112, -1, 115, -1, 152, -1, - 158, -1, 164, -1, 170, -1, 172, -1, 175, -1, - 176, -1, 181, -1, 182, -1, 193, -1, 208, -1, - 209, -1, 216, -1, 218, -1, 215, -1, 221, -1, - 222, -1, 224, -1, 234, -1, 236, -1, 246, -1, - 267, -1, 268, -1, 269, -1, 273, -1, 277, -1, - 280, -1, 282, -1, 299, -1, 298, -1, 301, -1, - 302, -1, 319, -1, 320, -1, 323, -1, 338, -1, - 360, -1, 364, -1, 377, -1, 384, -1, 388, -1, - 391, -1, 406, -1, 408, -1, 415, -1, 423, -1, - 424, -1, 429, -1, 431, -1, 435, -1, 457, -1, - 458, -1, 461, -1, 480, -1, 481, -1, 482, -1, - 483, -1, 484, -1, 485, -1, 486, -1, 487, -1, - 488, -1, 489, -1, 490, -1, 37, -1, 42, -1, - 49, -1, 55, -1, 81, -1, 83, -1, 89, -1, - 101, -1, 170, -1, 172, -1, 175, -1, 176, -1, - 193, -1, 208, -1, 221, -1, 222, -1, 224, -1, - 234, -1, 236, -1, 246, -1, 268, -1, 277, -1, - 299, -1, 301, -1, 320, -1, 360, -1, 377, -1, - 388, -1, 406, -1, 415, -1, 435, -1, 461, -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, 74, -1, 80, -1, 82, -1, - 93, -1, 100, -1, 117, -1, 119, -1, 126, -1, - 132, -1, 133, -1, 139, -1, 143, -1, 147, -1, - 159, -1, 161, -1, 166, -1, 168, -1, 171, -1, - 178, -1, 180, -1, 185, -1, 198, -1, 206, -1, - 217, -1, 219, -1, 231, -1, 232, -1, 237, -1, - 274, -1, 279, -1, 286, -1, 289, -1, 290, -1, - 294, -1, 295, -1, 313, -1, 314, -1, 315, -1, - 316, -1, 327, -1, 334, -1, 343, -1, 357, -1, - 376, -1, 393, -1, 410, -1, 413, -1, 421, -1, - 425, -1, 426, -1, 432, -1, 441, -1, 442, -1, - 446, -1, 451, -1, 459, -1, 469, -1, 470, -1, - 472, -1, 473, -1 -}; - -/* YYRLINE[YYN] -- source line where rule number YYN was defined. */ -static const yytype_uint16 yyrline[] = +/* YYRLINE[YYN] -- Source line where rule number YYN was defined. */ +static const yytype_int16 yyrline[] = { - 0, 494, 494, 510, 522, 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, 567, 569, 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, - 327, 335, 350, 358, 368, 378, 385, 392, 400, 407, - 418, 419, 424, 428, 433, 438, 446, 447, 452, 456, - 457, 458, 7, 13, 19, 25, 9, 13, 44, 45, - 46, 50, 51, 55, 59, 60, 64, 70, 75, 76, - 77, 78, 6, 15, 25, 35, 45, 55, 65, 75, - 85, 95, 106, 117, 127, 140, 141, 9, 23, 29, - 36, 42, 49, 59, 63, 71, 72, 73, 77, 86, - 95, 102, 103, 108, 120, 125, 150, 155, 160, 166, - 176, 186, 192, 203, 214, 229, 230, 236, 237, 242, - 243, 249, 250, 254, 255, 260, 262, 268, 269, 273, - 274, 277, 278, 283, 7, 16, 25, 46, 47, 50, - 54, 7, 14, 22, 9, 19, 29, 42, 43, 7, - 14, 31, 51, 55, 65, 69, 75, 76, 9, 17, - 29, 30, 34, 35, 36, 41, 42, 43, 48, 52, - 56, 60, 64, 68, 72, 76, 80, 84, 88, 92, - 97, 101, 105, 112, 113, 117, 118, 119, 2, 9, - 15, 21, 28, 35, 45, 46, 47, 2, 7, 21, - 36, 56, 57, 84, 85, 86, 87, 88, 89, 93, - 94, 99, 104, 105, 106, 107, 108, 113, 120, 121, - 122, 139, 146, 153, 163, 173, 185, 193, 202, 220, - 221, 225, 226, 230, 239, 262, 276, 283, 288, 290, - 292, 294, 297, 300, 301, 302, 303, 308, 312, 313, - 318, 325, 330, 331, 332, 333, 334, 335, 336, 337, - 343, 344, 348, 353, 360, 367, 374, 386, 387, 388, - 389, 393, 398, 399, 400, 405, 410, 411, 412, 413, - 414, 415, 420, 440, 469, 470, 474, 478, 479, 480, - 484, 488, 496, 497, 502, 503, 504, 508, 516, 517, - 522, 523, 527, 532, 536, 540, 545, 553, 554, 558, - 559, 563, 564, 570, 581, 594, 608, 622, 636, 650, - 673, 677, 684, 688, 696, 701, 708, 718, 719, 720, - 721, 722, 729, 736, 737, 742, 743, 9, 19, 29, - 39, 49, 59, 73, 74, 75, 76, 77, 78, 79, - 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, - 90, 95, 96, 97, 98, 99, 100, 105, 106, 111, - 112, 113, 118, 119, 120, 8, 20, 33, 46, 58, - 70, 86, 87, 91, 95, 7, 1, 30, 49, 61, - 62, 63, 67, 68, 73, 77, 82, 86, 94, 95, - 99, 100, 105, 106, 110, 111, 116, 117, 118, 119, - 120, 121, 126, 134, 138, 143, 144, 149, 153, 158, - 162, 166, 170, 174, 178, 182, 186, 190, 194, 198, - 202, 206, 210, 214, 218, 226, 231, 232, 233, 238, - 242, 47, 48, 52, 53, 68, 69, 76, 84, 92, - 100, 108, 116, 127, 128, 155, 160, 168, 184, 201, - 218, 235, 236, 255, 259, 263, 267, 271, 280, 290, - 299, 308, 318, 328, 339, 353, 370, 370, 374, 374, - 378, 378, 382, 388, 395, 399, 400, 404, 405, 419, - 426, 433, 443, 444, 447, 460, 461, 462, 466, 477, - 485, 490, 495, 500, 505, 513, 521, 526, 531, 538, - 539, 543, 544, 545, 549, 556, 557, 561, 562, 566, - 567, 568, 572, 573, 577, 578, 594, 595, 598, 607, - 618, 619, 620, 623, 624, 625, 629, 630, 631, 632, - 636, 637, 641, 643, 659, 661, 666, 669, 677, 681, - 685, 689, 693, 697, 704, 709, 716, 717, 721, 726, - 730, 734, 742, 749, 750, 755, 756, 760, 761, 766, - 768, 770, 775, 795, 796, 798, 803, 804, 808, 809, - 812, 813, 838, 839, 844, 848, 849, 853, 854, 858, - 859, 860, 861, 862, 866, 879, 886, 893, 900, 901, - 905, 906, 910, 911, 915, 916, 920, 921, 925, 926, - 930, 941, 942, 943, 944, 948, 949, 954, 955, 956, - 965, 971, 980, 981, 994, 995, 999, 1000, 1004, 1005, - 1011, 1017, 1025, 1034, 1042, 1051, 1060, 1064, 1069, 1079, - 1092, 1093, 1096, 1097, 1098, 1101, 1109, 1119, 1120, 1123, - 1131, 1140, 1144, 1151, 1152, 1156, 1165, 1169, 1194, 1198, - 1211, 1225, 1240, 1252, 1265, 1279, 1293, 1306, 1321, 1340, - 1346, 1351, 1357, 1364, 1365, 1373, 1377, 1381, 1387, 1394, - 1399, 1400, 1401, 1402, 1403, 1404, 1408, 1409, 1421, 1422, - 1427, 1434, 1441, 1448, 1480, 1491, 1504, 1509, 1510, 1513, - 1514, 1517, 1518, 1523, 1524, 1529, 1533, 1539, 1560, 1568, - 1581, 1584, 1588, 1588, 1591, 1592, 1594, 1599, 1606, 1611, - 1617, 1622, 1628, 1632, 1639, 1646, 1656, 1657, 1661, 1663, - 1666, 1670, 1671, 1672, 1673, 1674, 1675, 1680, 1700, 1701, - 1702, 1703, 1714, 1728, 1729, 1735, 1740, 1745, 1750, 1755, - 1760, 1765, 1770, 1776, 1782, 1788, 1795, 1817, 1826, 1830, - 1838, 1842, 1850, 1862, 1883, 1887, 1893, 1897, 1910, 1918, - 1928, 1930, 1932, 1934, 1936, 1938, 1943, 1944, 1951, 1960, - 1968, 1977, 1988, 1996, 1997, 1998, 2002, 2002, 2005, 2005, - 2008, 2008, 2011, 2011, 2014, 2014, 2017, 2017, 2020, 2020, - 2023, 2023, 2026, 2026, 2029, 2029, 2032, 2032, 2035, 2035, - 2038, 2040, 2042, 2044, 2046, 2048, 2050, 2052, 2054, 2056, - 2058, 2060, 2062, 2067, 2072, 2078, 2085, 2090, 2096, 2102, - 2133, 2135, 2137, 2145, 2160, 2162, 2164, 2166, 2168, 2170, - 2172, 2174, 2176, 2178, 2180, 2182, 2184, 2186, 2188, 2190, - 2193, 2195, 2197, 2200, 2202, 2204, 2206, 2208, 2213, 2218, - 2225, 2230, 2237, 2242, 2249, 2254, 2262, 2270, 2278, 2286, - 2304, 2312, 2320, 2328, 2336, 2344, 2352, 2356, 2372, 2380, - 2388, 2396, 2404, 2412, 2420, 2424, 2428, 2432, 2436, 2444, - 2452, 2460, 2468, 2488, 2510, 2521, 2528, 2542, 2550, 2558, - 2578, 2580, 2582, 2584, 2586, 2588, 2590, 2592, 2594, 2596, - 2598, 2600, 2602, 2604, 2606, 2608, 2610, 2612, 2614, 2616, - 2618, 2620, 2624, 2628, 2632, 2646, 2647, 2651, 2665, 2666, - 2667, 2674, 2678, 2682, 2685, 2696, 2701, 2703, 2714, 2738, - 2749, 2760, 2764, 2771, 2775, 2779, 2796, 2804, 2813, 2817, - 2824, 2832, 2840, 2851, 2871, 2907, 2918, 2919, 2926, 2932, - 2934, 2936, 2940, 2949, 2954, 2961, 2976, 2983, 2987, 2991, - 2995, 2999, 3009, 3018, 3040, 3041, 3045, 3046, 3047, 3051, - 3052, 3059, 3060, 3064, 3065, 3070, 3078, 3080, 3094, 3097, - 3124, 3125, 3128, 3129, 3137, 3145, 3153, 3162, 3172, 3190, - 3236, 3245, 3254, 3263, 3272, 3284, 3285, 3286, 3287, 3288, - 3302, 3303, 3306, 3307, 3311, 3321, 3322, 3326, 3327, 3331, - 3338, 3339, 3344, 3345, 3350, 3351, 3354, 3355, 3356, 3359, - 3360, 3363, 3364, 3365, 3366, 3367, 3368, 3369, 3370, 3371, - 3372, 3373, 3374, 3375, 3376, 3379, 3381, 3386, 3388, 3393, - 3395, 3397, 3399, 3401, 3403, 3405, 3407, 3421, 3423, 3428, - 3432, 3439, 3444, 3450, 3454, 3461, 3466, 3473, 3478, 3486, - 3490, 3496, 3500, 3509, 3520, 3521, 3525, 3529, 3536, 3537, - 3538, 3539, 3540, 3541, 3542, 3543, 3544, 3545, 3546, 3547, - 3548, 3549, 3559, 3563, 3570, 3577, 3578, 3594, 3598, 3603, - 3607, 3622, 3627, 3631, 3634, 3637, 3638, 3639, 3642, 3649, - 3659, 3673, 3674, 3678, 3689, 3690, 3693, 3694, 3697, 3701, - 3708, 3716, 3724, 3732, 3742, 3743, 3748, 3749, 3753, 3754, - 3755, 3759, 3768, 3776, 3784, 3793, 3808, 3809, 3814, 3815, - 3825, 3826, 3830, 3831, 3835, 3836, 3839, 3855, 3863, 3873, - 3874, 3877, 3878, 3881, 3885, 3886, 3890, 3891, 3894, 3895, - 3896, 3906, 3907, 3911, 3913, 3919, 3920, 3924, 3925, 3928, - 3939, 3942, 3953, 3957, 3961, 3973, 3977, 3986, 3993, 4031, - 4035, 4039, 4043, 4047, 4051, 4055, 4061, 4078, 4079, 4080, - 4083, 4084, 4085, 4088, 4089, 4090, 4093, 4094, 4097, 4099, - 4104, 4105, 4108, 4112, 4113, 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, 89, 95, 99, 103, 118, 125, 126, - 127, 131, 132, 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, - 16, 28, 35, 44, 45, 49, 50, 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, + 0, 500, 500, 516, 528, 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, 567, 568, 569, 570, 571, + 572, 573, 574, 575, 576, 578, 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, 327, 335, 350, 358, 368, 378, 385, + 392, 400, 407, 418, 419, 424, 428, 433, 438, 446, + 447, 452, 456, 457, 458, 7, 13, 19, 25, 9, + 13, 44, 45, 46, 50, 51, 55, 59, 60, 64, + 70, 75, 76, 77, 78, 6, 15, 25, 35, 45, + 55, 65, 75, 85, 95, 106, 117, 127, 140, 141, + 9, 23, 29, 36, 42, 49, 59, 63, 71, 72, + 73, 77, 86, 95, 102, 103, 108, 120, 125, 150, + 155, 160, 166, 176, 186, 192, 203, 214, 229, 230, + 236, 237, 242, 243, 249, 250, 254, 255, 260, 262, + 268, 269, 273, 274, 277, 278, 283, 7, 16, 25, + 46, 47, 50, 54, 7, 14, 22, 9, 19, 29, + 42, 43, 7, 17, 27, 40, 41, 45, 46, 47, + 51, 52, 7, 14, 31, 51, 55, 65, 69, 75, + 76, 9, 17, 29, 30, 34, 35, 36, 41, 42, + 43, 48, 52, 56, 60, 64, 68, 72, 76, 80, + 84, 88, 92, 97, 101, 105, 112, 113, 117, 118, + 119, 7, 16, 28, 29, 2, 9, 15, 21, 28, + 35, 45, 46, 47, 2, 7, 21, 36, 56, 57, + 84, 85, 86, 87, 88, 89, 93, 94, 99, 104, + 105, 106, 107, 108, 113, 120, 121, 122, 139, 146, + 153, 163, 173, 185, 193, 202, 220, 221, 225, 226, + 230, 239, 262, 276, 283, 288, 290, 292, 294, 297, + 300, 301, 302, 303, 308, 312, 313, 318, 325, 330, + 331, 332, 333, 334, 335, 336, 337, 343, 344, 348, + 353, 360, 367, 374, 386, 387, 388, 389, 393, 398, + 399, 400, 405, 410, 411, 412, 413, 414, 415, 420, + 440, 469, 470, 474, 478, 479, 480, 484, 488, 496, + 497, 502, 503, 504, 508, 516, 517, 522, 523, 527, + 532, 536, 540, 545, 553, 554, 558, 559, 563, 564, + 570, 581, 594, 608, 622, 636, 650, 673, 677, 684, + 688, 696, 701, 708, 718, 719, 720, 721, 722, 729, + 736, 737, 742, 743, 9, 19, 29, 39, 49, 59, + 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, + 83, 84, 85, 86, 87, 88, 89, 90, 95, 96, + 97, 98, 99, 100, 105, 106, 111, 112, 113, 118, + 119, 120, 8, 20, 33, 46, 58, 70, 86, 87, + 91, 95, 7, 1, 30, 49, 61, 62, 63, 67, + 68, 73, 77, 82, 86, 94, 95, 99, 100, 105, + 106, 110, 111, 116, 117, 118, 119, 120, 121, 126, + 134, 138, 143, 144, 149, 153, 158, 162, 166, 170, + 174, 178, 182, 186, 190, 194, 198, 202, 206, 210, + 214, 218, 226, 231, 232, 233, 234, 235, 241, 245, + 47, 48, 52, 53, 54, 72, 73, 80, 88, 96, + 104, 112, 120, 131, 132, 159, 164, 172, 188, 205, + 222, 239, 240, 259, 263, 267, 271, 275, 284, 294, + 303, 312, 322, 332, 343, 357, 374, 374, 378, 378, + 382, 382, 386, 392, 399, 403, 404, 408, 409, 423, + 430, 437, 447, 448, 451, 464, 465, 466, 470, 481, + 489, 494, 499, 504, 509, 517, 525, 530, 535, 542, + 543, 547, 548, 549, 553, 560, 561, 565, 566, 570, + 571, 572, 576, 577, 581, 582, 598, 599, 602, 611, + 622, 623, 624, 627, 628, 629, 633, 634, 635, 636, + 640, 641, 645, 647, 663, 665, 670, 673, 681, 685, + 689, 693, 697, 701, 708, 713, 720, 721, 725, 730, + 734, 738, 746, 753, 754, 759, 760, 764, 765, 770, + 772, 774, 779, 799, 800, 802, 807, 808, 812, 813, + 816, 817, 842, 843, 848, 852, 853, 857, 858, 862, + 863, 864, 865, 866, 870, 883, 890, 897, 904, 905, + 909, 910, 914, 915, 919, 920, 924, 925, 929, 930, + 934, 945, 946, 947, 948, 952, 953, 958, 959, 960, + 969, 975, 984, 985, 998, 999, 1003, 1004, 1008, 1009, + 1015, 1021, 1029, 1038, 1046, 1055, 1064, 1068, 1073, 1083, + 1096, 1097, 1100, 1101, 1102, 1105, 1113, 1123, 1124, 1127, + 1135, 1144, 1148, 1155, 1156, 1160, 1169, 1173, 1198, 1202, + 1215, 1229, 1244, 1256, 1269, 1283, 1297, 1310, 1325, 1344, + 1350, 1355, 1361, 1368, 1369, 1377, 1381, 1385, 1391, 1398, + 1403, 1404, 1405, 1406, 1407, 1408, 1412, 1413, 1425, 1426, + 1431, 1438, 1445, 1452, 1484, 1495, 1508, 1513, 1514, 1517, + 1518, 1521, 1522, 1527, 1528, 1533, 1537, 1543, 1564, 1572, + 1585, 1588, 1592, 1592, 1595, 1596, 1598, 1603, 1610, 1615, + 1621, 1626, 1632, 1636, 1643, 1650, 1660, 1661, 1665, 1667, + 1670, 1674, 1675, 1676, 1677, 1678, 1679, 1684, 1704, 1705, + 1706, 1707, 1718, 1732, 1733, 1739, 1744, 1749, 1754, 1759, + 1764, 1769, 1774, 1780, 1786, 1792, 1799, 1821, 1830, 1834, + 1842, 1846, 1854, 1866, 1887, 1891, 1897, 1901, 1914, 1922, + 1932, 1934, 1936, 1938, 1940, 1942, 1947, 1948, 1955, 1964, + 1972, 1981, 1992, 2000, 2001, 2002, 2006, 2006, 2009, 2009, + 2012, 2012, 2015, 2015, 2018, 2018, 2021, 2021, 2024, 2024, + 2027, 2027, 2030, 2030, 2033, 2033, 2036, 2036, 2039, 2039, + 2042, 2044, 2046, 2048, 2050, 2052, 2054, 2056, 2058, 2060, + 2062, 2064, 2066, 2071, 2076, 2082, 2089, 2094, 2100, 2106, + 2137, 2139, 2141, 2149, 2164, 2166, 2168, 2170, 2172, 2174, + 2176, 2178, 2180, 2182, 2184, 2186, 2188, 2190, 2192, 2194, + 2197, 2199, 2201, 2204, 2206, 2208, 2210, 2212, 2217, 2222, + 2229, 2234, 2241, 2246, 2253, 2258, 2266, 2274, 2282, 2290, + 2308, 2316, 2324, 2332, 2340, 2348, 2356, 2360, 2376, 2384, + 2392, 2400, 2408, 2416, 2424, 2428, 2432, 2436, 2440, 2448, + 2456, 2464, 2472, 2492, 2514, 2525, 2532, 2546, 2554, 2562, + 2582, 2584, 2586, 2588, 2590, 2592, 2594, 2596, 2598, 2600, + 2602, 2604, 2606, 2608, 2610, 2612, 2614, 2616, 2618, 2620, + 2622, 2624, 2628, 2632, 2636, 2650, 2651, 2655, 2669, 2670, + 2671, 2678, 2682, 2686, 2689, 2700, 2705, 2707, 2718, 2742, + 2753, 2764, 2768, 2775, 2779, 2783, 2800, 2808, 2817, 2821, + 2828, 2836, 2844, 2855, 2875, 2911, 2922, 2923, 2930, 2936, + 2938, 2940, 2944, 2953, 2958, 2965, 2980, 2987, 2991, 2995, + 2999, 3003, 3013, 3022, 3044, 3045, 3049, 3050, 3051, 3055, + 3056, 3063, 3064, 3068, 3069, 3074, 3082, 3084, 3098, 3101, + 3128, 3129, 3132, 3133, 3141, 3149, 3157, 3166, 3176, 3194, + 3240, 3249, 3258, 3267, 3276, 3288, 3289, 3290, 3291, 3292, + 3306, 3307, 3310, 3311, 3315, 3325, 3326, 3330, 3331, 3335, + 3342, 3343, 3348, 3349, 3354, 3355, 3358, 3359, 3360, 3363, + 3364, 3367, 3368, 3369, 3370, 3371, 3372, 3373, 3374, 3375, + 3376, 3377, 3378, 3379, 3380, 3383, 3385, 3390, 3392, 3397, + 3399, 3401, 3403, 3405, 3407, 3409, 3411, 3425, 3427, 3432, + 3436, 3443, 3448, 3454, 3458, 3465, 3470, 3477, 3482, 3490, + 3494, 3500, 3504, 3513, 3524, 3525, 3529, 3533, 3540, 3541, + 3542, 3543, 3544, 3545, 3546, 3547, 3548, 3549, 3550, 3551, + 3552, 3553, 3563, 3567, 3574, 3581, 3582, 3598, 3602, 3607, + 3611, 3626, 3631, 3635, 3638, 3641, 3642, 3643, 3646, 3653, + 3663, 3677, 3678, 3682, 3693, 3694, 3697, 3698, 3701, 3705, + 3712, 3720, 3728, 3736, 3746, 3747, 3752, 3753, 3757, 3758, + 3759, 3763, 3772, 3780, 3788, 3797, 3812, 3813, 3818, 3819, + 3829, 3830, 3834, 3835, 3839, 3840, 3843, 3859, 3867, 3877, + 3878, 3881, 3882, 3885, 3889, 3890, 3894, 3895, 3898, 3899, + 3900, 3910, 3911, 3915, 3917, 3923, 3924, 3928, 3929, 3932, + 3943, 3946, 3957, 3961, 3965, 3977, 3981, 3990, 3997, 4035, + 4039, 4043, 4047, 4051, 4055, 4059, 4065, 4082, 4083, 4084, + 4087, 4088, 4089, 4092, 4093, 4094, 4097, 4098, 4101, 4103, + 4108, 4109, 4112, 4116, 4117, 7, 18, 19, 23, 24, + 25, 26, 27, 28, 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, 15, 26, 27, + 30, 31, 32, 33, 34, 35, 36, 37, 38, 39, + 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, 89, 95, + 99, 103, 118, 125, 126, 127, 131, 132, 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, 16, 28, 35, 42, 51, + 52, 56, 57, 2, 7, 12, 17, 26, 33, 43, + 44, 51, 3, 10, 17, 24, 31, 38, 45, 52, + 61, 61, 63, 63, 65, 65, 67, 68, 72, 73, + 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, @@ -2676,18 +1867,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, 74, 74, 74, 74, 74, 75, + 74, 74, 74, 74, 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, 75, 76, 76, 76, 76, 76, 76, 76, - 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, + 75, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, 76, - 76, 77, 77, 77, 77, 77, 77, 77, 77, 77, + 76, 76, 76, 76, 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, 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, @@ -2695,44 +1886,52 @@ 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, 78, 78, 78, 78, - 78, 79, 79, 79, 79, 79, 79, 79, 79, 79, + 78, 78, 78, 78, 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, 79, 80, 80, 80, 80, 80, 80, 80, + 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 }; #endif -#if YYDEBUG || YYERROR_VERBOSE || YYTOKEN_TABLE +/** Accessing symbol of state STATE. */ +#define YY_ACCESSING_SYMBOL(State) YY_CAST (yysymbol_kind_t, yystos[State]) + +#if YYDEBUG || 0 +/* The user-facing name of the symbol whose (internal) number is + YYSYMBOL. No bounds checking. */ +static const char *yysymbol_name (yysymbol_kind_t yysymbol) YY_ATTRIBUTE_UNUSED; + /* YYTNAME[SYMBOL-NUM] -- String name of the symbol SYMBOL-NUM. First, the terminals, then, starting at YYNTOKENS, nonterminals. */ static const char *const yytname[] = { - "$end", "error", "$undefined", "IDENT", "FCONST", "SCONST", "BCONST", - "XCONST", "Op", "ICONST", "PARAM", "TYPECAST", "DOT_DOT", "COLON_EQUALS", - "EQUALS_GREATER", "INTEGER_DIVISION", "POWER_OF", "LAMBDA_ARROW", - "DOUBLE_ARROW", "LESS_EQUALS", "GREATER_EQUALS", "NOT_EQUALS", "ABORT_P", - "ABSOLUTE_P", "ACCESS", "ACTION", "ADD_P", "ADMIN", "AFTER", "AGGREGATE", - "ALL", "ALSO", "ALTER", "ALWAYS", "ANALYSE", "ANALYZE", "AND", "ANTI", - "ANY", "ARRAY", "AS", "ASC_P", "ASOF", "ASSERTION", "ASSIGNMENT", - "ASYMMETRIC", "AT", "ATTACH", "ATTRIBUTE", "AUTHORIZATION", "BACKWARD", - "BEFORE", "BEGIN_P", "BETWEEN", "BIGINT", "BINARY", "BIT", "BOOLEAN_P", - "BOTH", "BY", "CACHE", "CALL_P", "CALLED", "CASCADE", "CASCADED", "CASE", - "CAST", "CATALOG_P", "CENTURIES_P", "CENTURY_P", "CHAIN", "CHAR_P", - "CHARACTER", "CHARACTERISTICS", "CHECK_P", "CHECKPOINT", "CLASS", - "CLOSE", "CLUSTER", "COALESCE", "COLLATE", "COLLATION", "COLUMN", - "COLUMNS", "COMMENT", "COMMENTS", "COMMIT", "COMMITTED", "COMPRESSION", - "CONCURRENTLY", "CONFIGURATION", "CONFLICT", "CONNECTION", "CONSTRAINT", - "CONSTRAINTS", "CONTENT_P", "CONTINUE_P", "CONVERSION_P", "COPY", "COST", - "CREATE_P", "CROSS", "CSV", "CUBE", "CURRENT_P", "CURSOR", "CYCLE", - "DATA_P", "DATABASE", "DAY_P", "DAYS_P", "DEALLOCATE", "DEC", "DECADE_P", + "\"end of file\"", "error", "\"invalid token\"", "IDENT", "FCONST", + "SCONST", "BCONST", "XCONST", "Op", "ICONST", "PARAM", "TYPECAST", + "DOT_DOT", "COLON_EQUALS", "EQUALS_GREATER", "INTEGER_DIVISION", + "POWER_OF", "LAMBDA_ARROW", "DOUBLE_ARROW", "LESS_EQUALS", + "GREATER_EQUALS", "NOT_EQUALS", "ABORT_P", "ABSOLUTE_P", "ACCESS", + "ACTION", "ADD_P", "ADMIN", "AFTER", "AGGREGATE", "ALL", "ALSO", "ALTER", + "ALWAYS", "ANALYSE", "ANALYZE", "AND", "ANTI", "ANY", "ARRAY", "AS", + "ASC_P", "ASOF", "ASSERTION", "ASSIGNMENT", "ASYMMETRIC", "AT", "ATTACH", + "ATTRIBUTE", "AUTHORIZATION", "BACKWARD", "BEFORE", "BEGIN_P", "BETWEEN", + "BIGINT", "BINARY", "BIT", "BOOLEAN_P", "BOTH", "BY", "CACHE", "CALL_P", + "CALLED", "CASCADE", "CASCADED", "CASE", "CAST", "CATALOG_P", + "CENTURIES_P", "CENTURY_P", "CHAIN", "CHAR_P", "CHARACTER", + "CHARACTERISTICS", "CHECK_P", "CHECKPOINT", "CLASS", "CLOSE", "CLUSTER", + "COALESCE", "COLLATE", "COLLATION", "COLUMN", "COLUMNS", "COMMENT", + "COMMENTS", "COMMIT", "COMMITTED", "COMPRESSION", "CONCURRENTLY", + "CONFIGURATION", "CONFLICT", "CONNECTION", "CONSTRAINT", "CONSTRAINTS", + "CONTENT_P", "CONTINUE_P", "CONVERSION_P", "COPY", "COST", "CREATE_P", + "CROSS", "CSV", "CUBE", "CURRENT_P", "CURSOR", "CYCLE", "DATA_P", + "DATABASE", "DAY_P", "DAYS_P", "DEALLOCATE", "DEC", "DECADE_P", "DECADES_P", "DECIMAL_P", "DECLARE", "DEFAULT", "DEFAULTS", "DEFERRABLE", "DEFERRED", "DEFINER", "DELETE_P", "DELIMITER", "DELIMITERS", "DEPENDS", "DESC_P", "DESCRIBE", "DETACH", "DICTIONARY", "DISABLE_P", "DISCARD", @@ -2763,32 +1962,33 @@ static const char *const yytname[] = "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", + "PARTIAL", "PARTITION", "PASSING", "PASSWORD", "PERCENT", "PERSISTENT", + "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", "WEEK_P", "WEEKS_P", "WHEN", "WHERE", + "SAVEPOINT", "SCHEMA", "SCHEMAS", "SCOPE", "SCROLL", "SEARCH", + "SECOND_P", "SECONDS_P", "SECRET", "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", "WEEK_P", "WEEKS_P", "WHEN", "WHERE", "WHITESPACE_P", "WINDOW", "WITH", "WITHIN", "WITHOUT", "WORK", "WRAPPER", "WRITE_P", "XML_P", "XMLATTRIBUTES", "XMLCONCAT", "XMLELEMENT", "XMLEXISTS", "XMLFOREST", "XMLNAMESPACES", "XMLPARSE", "XMLPI", @@ -2797,45 +1997,62 @@ static const char *const yytname[] = "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", "alter_table_cmds", "alter_generic_options", - "opt_set_data", "DeallocateStmt", "qualified_name", "ColId", - "ColIdOrString", "Sconst", "indirection", "indirection_el", "attr_name", - "ColLabel", "RenameStmt", "opt_column", "InsertStmt", "insert_rest", - "insert_target", "opt_by_name_or_position", "opt_conf_expr", - "opt_with_clause", "insert_column_item", "set_clause", "opt_or_action", - "opt_on_conflict", "index_elem", "returning_clause", "override_kind", - "set_target_list", "opt_collate", "opt_class", "insert_column_list", - "set_clause_list", "set_clause_list_opt_comma", "index_params", - "set_target", "CreateTypeStmt", "opt_enum_val_list", "enum_val_list", - "PragmaStmt", "CreateSeqStmt", "OptSeqOptList", "ExecuteStmt", + "AlterObjectSchemaStmt", "AlterSeqStmt", "SeqOptList", "opt_with", + "NumericOnly", "SeqOptElem", "opt_by", "SignedIconst", "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", + "alter_table_cmds", "alter_generic_options", "opt_set_data", + "AnalyzeStmt", "AttachStmt", "DetachStmt", "opt_database", + "opt_database_alias", "CallStmt", "CheckPointStmt", "opt_col_id", + "CommentOnStmt", "comment_value", "comment_on_type_any_name", + "qualified_name", "ColId", "ColIdOrString", "Sconst", "indirection", + "indirection_el", "attr_name", "ColLabel", "CopyStmt", + "copy_database_flag", "copy_from", "copy_delimiter", + "copy_generic_opt_arg_list", "opt_using", "opt_as", "opt_program", + "copy_options", "copy_generic_opt_arg", "copy_generic_opt_elem", + "opt_oids", "copy_opt_list", "opt_binary", "copy_opt_item", + "copy_generic_opt_arg_list_item", "copy_file_name", + "copy_generic_opt_list", "CreateStmt", "ConstraintAttributeSpec", + "def_arg", "OptParenthesizedSeqOptList", "generic_option_arg", + "key_action", "ColConstraint", "ColConstraintElem", + "GeneratedColumnType", "opt_GeneratedColumnType", + "GeneratedConstraintElem", "generic_option_elem", "key_update", + "key_actions", "OnCommitOption", "reloptions", "opt_no_inherit", + "TableConstraint", "TableLikeOption", "reloption_list", "ExistingIndex", + "ConstraintAttr", "OptWith", "definition", "TableLikeOptionList", + "generic_option_name", "ConstraintAttributeElem", "columnDef", + "def_list", "index_name", "TableElement", "def_elem", "opt_definition", + "OptTableElementList", "columnElem", "opt_column_list", "ColQualList", + "key_delete", "reloption_elem", "columnList", "columnList_opt_comma", + "func_type", "ConstraintElem", "TableElementList", "key_match", + "TableLikeClause", "OptTemp", "generated_when", "CreateAsStmt", + "opt_with_data", "create_as_target", "unreserved_keyword", + "col_name_keyword", "func_name_keyword", "type_name_keyword", + "other_keyword", "type_func_name_keyword", "reserved_keyword", + "CreateFunctionStmt", "macro_alias", "param_list", "CreateSchemaStmt", + "OptSchemaEltList", "schema_stmt", "CreateSecretStmt", "opt_secret_name", + "opt_persist", "opt_storage_specifier", "CreateSeqStmt", "OptSeqOptList", + "CreateTypeStmt", "opt_enum_val_list", "enum_val_list", "DeallocateStmt", + "DeleteStmt", "relation_expr_opt_alias", "where_or_current_clause", + "using_clause", "DropStmt", "drop_type_any_name", "drop_type_name", + "any_name_list", "opt_drop_behavior", "drop_type_name_on_any_name", + "DropSecretStmt", "opt_storage_drop_specifier", "ExecuteStmt", "execute_param_expr", "execute_param_list", "execute_param_clause", - "AlterSeqStmt", "SeqOptList", "opt_with", "NumericOnly", "SeqOptElem", - "opt_by", "SignedIconst", "TransactionStmt", "opt_transaction", - "UseStmt", "CreateStmt", "ConstraintAttributeSpec", "def_arg", - "OptParenthesizedSeqOptList", "generic_option_arg", "key_action", - "ColConstraint", "ColConstraintElem", "GeneratedColumnType", - "opt_GeneratedColumnType", "GeneratedConstraintElem", - "generic_option_elem", "key_update", "key_actions", "OnCommitOption", - "reloptions", "opt_no_inherit", "TableConstraint", "TableLikeOption", - "reloption_list", "ExistingIndex", "ConstraintAttr", "OptWith", - "definition", "TableLikeOptionList", "generic_option_name", - "ConstraintAttributeElem", "columnDef", "def_list", "index_name", - "TableElement", "def_elem", "opt_definition", "OptTableElementList", - "columnElem", "opt_column_list", "ColQualList", "key_delete", - "reloption_elem", "columnList", "columnList_opt_comma", "func_type", - "ConstraintElem", "TableElementList", "key_match", "TableLikeClause", - "OptTemp", "generated_when", "DropStmt", "drop_type_any_name", - "drop_type_name", "any_name_list", "opt_drop_behavior", - "drop_type_name_on_any_name", "CreateFunctionStmt", "macro_alias", - "param_list", "UpdateStmt", "CopyStmt", "copy_database_flag", - "copy_from", "copy_delimiter", "copy_generic_opt_arg_list", "opt_using", - "opt_as", "opt_program", "copy_options", "copy_generic_opt_arg", - "copy_generic_opt_elem", "opt_oids", "copy_opt_list", "opt_binary", - "copy_opt_item", "copy_generic_opt_arg_list_item", "copy_file_name", - "copy_generic_opt_list", "SelectStmt", "select_with_parens", + "ExplainStmt", "opt_verbose", "explain_option_arg", "ExplainableStmt", + "NonReservedWord", "NonReservedWord_or_Sconst", "explain_option_list", + "analyze_keyword", "opt_boolean_or_string", "explain_option_elem", + "explain_option_name", "ExportStmt", "ImportStmt", "IndexStmt", + "access_method", "access_method_clause", "opt_concurrently", + "opt_index_name", "opt_reloptions", "opt_unique", "InsertStmt", + "insert_rest", "insert_target", "opt_by_name_or_position", + "opt_conf_expr", "opt_with_clause", "insert_column_item", "set_clause", + "opt_or_action", "opt_on_conflict", "index_elem", "returning_clause", + "override_kind", "set_target_list", "opt_collate", "opt_class", + "insert_column_list", "set_clause_list", "set_clause_list_opt_comma", + "index_params", "set_target", "LoadStmt", "file_name", "repo_path", + "PragmaStmt", "PrepareStmt", "prep_type_clause", "PreparableStmt", + "RenameStmt", "opt_column", "SelectStmt", "select_with_parens", "select_no_parens", "select_clause", "opt_select", "simple_select", "value_or_values", "pivot_keyword", "unpivot_keyword", "pivot_column_entry", "pivot_column_list_internal", "pivot_column_list", @@ -2900,9173 +2117,9067 @@ static const char *const yytname[] = "name_list_opt_comma_opt_bracket", "name", "func_name", "AexprConst", "Iconst", "type_function_name", "function_name_token", "type_name_token", "any_name", "attrs", "opt_name_list", "param_name", "ColLabelOrString", - "PrepareStmt", "prep_type_clause", "PreparableStmt", "CreateSchemaStmt", - "OptSchemaEltList", "schema_stmt", "IndexStmt", "access_method", - "access_method_clause", "opt_concurrently", "opt_index_name", - "opt_reloptions", "opt_unique", "AlterObjectSchemaStmt", - "CheckPointStmt", "opt_col_id", "ExportStmt", "ImportStmt", - "ExplainStmt", "opt_verbose", "explain_option_arg", "ExplainableStmt", - "NonReservedWord", "NonReservedWord_or_Sconst", "explain_option_list", - "analyze_keyword", "opt_boolean_or_string", "explain_option_elem", - "explain_option_name", "VariableSetStmt", "set_rest", "generic_set", - "var_value", "zone_value", "var_list", "LoadStmt", "file_name", - "repo_path", "VacuumStmt", "vacuum_option_elem", "opt_full", - "vacuum_option_list", "opt_freeze", "DeleteStmt", - "relation_expr_opt_alias", "where_or_current_clause", "using_clause", - "AnalyzeStmt", "AttachStmt", "DetachStmt", "opt_database", - "opt_database_alias", "VariableResetStmt", "generic_reset", "reset_rest", - "VariableShowStmt", "show_or_describe", "opt_tables", "var_name", - "table_id", "CallStmt", "ViewStmt", "opt_check_option", "CreateAsStmt", - "opt_with_data", "create_as_target", "unreserved_keyword", - "col_name_keyword", "func_name_keyword", "type_name_keyword", - "other_keyword", "type_func_name_keyword", "reserved_keyword", 0 + "TransactionStmt", "opt_transaction", "UpdateStmt", "UseStmt", + "VacuumStmt", "vacuum_option_elem", "opt_full", "vacuum_option_list", + "opt_freeze", "VariableResetStmt", "generic_reset", "reset_rest", + "VariableSetStmt", "set_rest", "generic_set", "var_value", "zone_value", + "var_list", "VariableShowStmt", "describe_or_desc", "show_or_describe", + "opt_tables", "var_name", "table_id", "ViewStmt", "opt_check_option", YY_NULLPTR }; + +static const char * +yysymbol_name (yysymbol_kind_t yysymbol) +{ + return yytname[yysymbol]; +} #endif -# ifdef YYPRINT -/* YYTOKNUM[YYLEX-NUM] -- Internal token number corresponding to - token YYLEX-NUM. */ -static const yytype_uint16 yytoknum[] = +#define YYPACT_NINF (-3054) + +#define yypact_value_is_default(Yyn) \ + ((Yyn) == YYPACT_NINF) + +#define YYTABLE_NINF (-2041) + +#define yytable_value_is_error(Yyn) \ + ((Yyn) == YYTABLE_NINF) + +/* YYPACT[STATE-NUM] -- Index in YYTABLE of the portion describing + STATE-NUM. */ +static const int yypact[] = { - 0, 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, 487, 488, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 510, 511, 512, 513, 514, - 515, 516, 517, 518, 519, 520, 521, 522, 523, 524, - 525, 526, 527, 528, 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, 566, 567, 568, 569, 570, 571, 572, 573, 574, - 575, 576, 577, 578, 579, 580, 581, 582, 583, 584, - 585, 586, 587, 588, 589, 590, 591, 592, 593, 594, - 595, 596, 597, 598, 599, 600, 601, 602, 603, 604, - 605, 606, 607, 608, 609, 610, 611, 612, 613, 614, - 615, 616, 617, 618, 619, 620, 621, 622, 623, 624, - 625, 626, 627, 628, 629, 630, 631, 632, 633, 634, - 635, 636, 637, 638, 639, 640, 641, 642, 643, 644, - 645, 646, 647, 648, 649, 650, 651, 652, 653, 654, - 655, 656, 657, 658, 659, 660, 661, 662, 663, 664, - 665, 666, 667, 668, 669, 670, 671, 672, 673, 674, - 675, 676, 677, 678, 679, 680, 681, 682, 683, 684, - 685, 686, 687, 688, 689, 690, 691, 692, 693, 694, - 695, 696, 697, 698, 699, 700, 701, 702, 703, 704, - 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, 742, 743, 744, - 745, 746, 747, 748, 749, 750, 751, 752, 60, 62, - 61, 753, 43, 45, 42, 47, 37, 94, 754, 91, - 93, 40, 41, 46, 59, 44, 35, 36, 63, 123, - 125, 58 + 6730, 369, 821, -3054, -3054, 278, 369, 49852, 64687, 414, + 369, 134, 2149, 51832, -3054, -3054, 46387, 8014, 369, 54802, + 72023, 324, 355, 31638, 654, 55297, 55297, -3054, -3054, -3054, + 64687, 54802, 55792, 369, 731, 65182, -3054, 369, 34113, 52327, + 417, -3054, 54802, 40, 97, 56287, 54802, 1995, 842, 340, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, 413, -3054, -3054, -3054, -3054, 111, + -3054, -3054, -3054, -3054, -3054, 151, -3054, 128, 154, 31638, + 31638, 1561, 384, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, 33618, -3054, -3054, -3054, -3054, 56782, 54802, 57277, + 52822, 57772, -3054, 671, 950, -3054, 167, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, 178, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, 186, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, 192, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, 638, 200, -3054, -3054, + -3054, 549, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + 1395, -3054, -3054, 865, 2206, 54802, 830, 836, 689, -3054, + 58267, -3054, 696, 54802, -3054, -3054, 806, 720, 884, -3054, + -3054, 53317, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, 46882, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, 854, -3054, -3054, + 727, -3054, 107, -3054, -3054, 726, 719, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, 795, -3054, -3054, -3054, + 798, 65677, 58762, 59257, -3054, 682, 1360, 6152, 72041, 30646, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, 413, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, 55297, 64687, 55297, 757, 776, 1072, 786, 32133, + 790, 34609, 794, 825, 1154, 872, 886, 898, 906, 97, + 31142, 787, 638, -3054, 59752, 59752, -57, 197, -3054, 59752, + 60247, -3054, 868, -3054, 950, -3054, -3054, -3054, 1262, -3054, + 135, 926, -3054, 60742, 60742, 60742, 965, 1241, -3054, -3054, + -3054, 958, -3054, -3054, 1197, 20385, 20385, 66172, 66172, 950, + 66172, 997, -3054, -3054, 73, -3054, -3054, -3054, 1561, 980, + 638, -3054, -3054, 52327, -3054, -3054, 294, 1336, 20385, 54802, + 994, -3054, 1051, 994, 1054, 1056, 1061, -3054, 6730, -3054, + 54802, 1414, 1279, 52327, 749, 749, 1533, 749, 622, 628, + 3173, 3454, -3054, 947, -3054, 1076, 1201, 1097, 1416, -3054, + 980, 1472, 905, 1293, 1519, 2639, 1538, 1035, 1544, 1077, + 1552, 1639, 20385, 47377, 638, -3054, 11564, 20385, -3054, -3054, + -3054, 1305, -3054, -3054, -3054, -3054, -3054, 54802, 64687, 1196, + 1222, -3054, -3054, -3054, -3054, 93, 1464, -3054, 1711, -3054, + -3054, 1287, 61237, 61732, 62227, 62722, 66667, 1656, -3054, -3054, + 1610, -3054, -3054, -3054, 1284, -3054, -3054, -3054, 226, 67162, + 1616, 1254, 122, -3054, 1623, 228, -3054, 1629, 1495, 15185, + -3054, 1443, -3054, -3054, -3054, 97, -3054, -3054, -3054, 629, + -3054, -3054, 43244, 72041, 1376, 1292, -3054, 1636, 20385, 20385, + 1296, 3361, 59752, 60247, 20385, 54802, -3054, 20385, 25065, 1298, + 20385, 20385, 12585, 20385, 29656, 59752, 197, 1294, -3054, 586, + 54802, 1299, -3054, 1396, 1396, 731, 31638, 1597, -3054, 880, + 1593, 1520, -3054, 31638, 1520, 1479, 1306, 1598, 1520, -3054, + 299, 1600, 1396, 35104, 1307, -3054, 1396, 1529, -3054, -3054, + 55297, 20385, 15185, 69637, 1783, -3054, -3054, -3054, -3054, 1591, + 64687, 1313, -3054, -3054, -3054, -3054, -3054, -3054, 822, 1823, + 174, 1824, 20385, 174, 174, 1316, 204, 204, -3054, 1508, + 1319, -3054, 210, 1321, 1322, 1832, 1833, 175, 150, 737, + 174, 20385, -3054, 204, 1327, 1837, 1329, 1839, 183, 195, + -3054, 211, 20385, 20385, 20385, 1697, 20385, 10524, 1838, 47377, + -3054, 54802, 709, -3054, 638, 1334, 950, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, 1335, -3054, 185, 7336, -3054, -3054, + -3054, -3054, 1374, -3054, -3054, -3054, -3054, 1554, 20385, -3054, + -3054, 1338, 1597, -3054, 212, -3054, -3054, 1597, -3054, -3054, + -3054, -3054, -3054, 233, 1748, 20385, 20385, 64687, 638, -3054, + 67657, -3054, -3054, -3054, -3054, -3054, -3054, 797, -3054, 413, + 44932, 1339, 1343, 994, 54802, 54802, 1828, -3054, -3054, -3054, + -3054, 1051, 52327, 209, 1644, 1478, -3054, -3054, 1561, 1561, + 15705, 891, 339, 710, 16225, 20905, 1706, 1583, 241, 607, + 1707, -3054, 1594, 1822, 25065, 20385, 20385, 622, 628, 20385, + -3054, -3054, -3054, 1647, 54802, 50347, 745, 837, 1371, 1459, + 1375, 52, 1795, -3054, 1372, -3054, 1470, 54802, 71572, 231, + -3054, 1840, 231, 231, 247, 1851, 1484, 259, 1649, 57, + -51, 2616, -3054, 1372, 52327, 159, 569, 1372, 54802, 1490, + 571, 1372, 1812, 64687, 1292, 41096, 1398, -3054, -3054, -3054, + 163, 15185, -3054, 911, 1117, 1231, 322, 208, 1250, 1272, + 15185, 1304, 1405, 173, 1471, 1488, 1505, 1531, 1540, 1585, + 1599, 1634, 145, 1648, 1671, 1688, 1701, 1703, 1705, -3054, + 1713, 188, 1715, 218, 15185, 1729, -3054, 190, -3054, 44932, + -8, -3054, -3054, 1736, 45021, -3054, 1702, 1496, 1497, 64687, + 1451, 54802, 867, 1555, 1777, 1831, 70117, 54802, 1660, 2616, + 1661, 1423, 1902, 1670, 1222, 1672, 1435, 1678, -3054, 1756, + -3054, 68152, 47377, -3054, -3054, -3054, -3054, -3054, 1807, 1789, + 64687, 47377, 1450, -3054, -3054, 64687, -3054, 54802, 54802, -3054, + 54802, 64687, -3054, 801, 44932, 1956, 117, 72041, 48862, -3054, + -3054, -3054, -3054, 425, 985, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, 950, 47377, -3054, 3660, 55297, 43862, + 1458, 20385, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + 1460, 1805, -3054, -3054, 6524, 1467, 43902, 1469, 25065, 25065, + 638, 1527, -3054, -3054, 25065, 1475, 49357, 43775, 1461, 1476, + 44250, 16745, 20385, 16745, 16745, 44337, -3054, 1477, 44414, 59752, + 1473, 54802, 53812, -3054, -3054, -3054, 20385, 20385, 197, 54307, + 1512, -3054, 31638, -3054, 1754, 31638, -3054, -3054, 1896, -3054, + 31638, 1762, 20385, 31638, -3054, 1721, 1723, 1480, 31638, -3054, + 54802, 1498, 54802, -3054, -3054, -3054, -3054, -3054, 44932, -3054, + 1487, 811, 1500, -3054, -3054, -3054, -3054, -3054, 1557, -3054, + 1557, 1557, -3054, -3054, -3054, -3054, 1509, 1509, 1511, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, 1517, 737, -3054, 1557, -3054, 1509, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, 71572, -3054, -3054, -3054, -3054, + 643, 681, -3054, 1525, -3054, -3054, -3054, -3054, 1528, -3054, + 1501, 1985, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, 6014, 823, 1509, 5103, -3054, -3054, 20385, 20385, -3054, + -3054, 1530, 44932, 1569, -3054, -3054, 20385, 20385, -3054, -3054, + -3054, -3054, 2017, -3054, 20385, 1557, 1557, -3054, 45363, -3054, + 39561, 17265, 1619, 1620, 2017, -3054, 2017, -3054, 45363, 2037, + 2037, 37579, -3054, 1698, 44501, -3054, 1536, 1503, 7998, 1537, + -3054, -3054, 1532, -3054, 1539, 1535, 41759, 164, 638, 638, + 20385, -3054, 2017, 20385, 7141, 7141, -3054, 344, 69637, 20385, + 20385, 20385, 20385, 20385, 20385, 20385, 20385, 45892, 1628, 162, + 64687, 20385, 20385, 1545, 1058, -3054, 20385, 1781, -3054, 1553, + 20385, 1638, 235, 20385, 20385, 20385, 20385, 20385, 20385, 20385, + 20385, 20385, -3054, -3054, 27646, 353, 724, 1889, 1909, 35, + 295, 20385, 1901, 11564, -3054, 1901, -3054, -3054, -3054, -3054, + -3054, 213, -3054, -3054, 1487, 1487, -3054, 64687, -3054, 54802, + 294, 51337, 20385, -3054, -3054, 1562, 1568, 193, -3054, 1630, + -3054, -3054, 54802, 38074, 1869, -3054, 367, 1572, -3054, 43736, + 1830, 1869, 1561, -3054, -3054, 26105, 1708, 1872, 1813, -3054, + -3054, 1794, 1796, -3054, 1589, 45154, 21425, 21425, -3054, 656, + 44932, 1391, -3054, -3054, -3054, -3054, -3054, -3054, 781, -3054, + 54802, 225, 35599, -3054, 1595, 96, -3054, 4021, 1936, 1900, + 1706, 607, 1611, -3054, 1266, 1614, 68647, 54802, 1899, 1858, + 1907, -60, 69637, -3054, -3054, -3054, -3054, 54802, 64687, 63217, + 69142, 47872, 54802, 47377, -3054, -3054, -3054, -3054, 54802, 1094, + 54802, 2790, -3054, -3054, -3054, -3054, 231, -3054, -3054, -3054, + -3054, -3054, 64687, 54802, -3054, -3054, 231, 64687, 54802, 231, + -3054, 1267, 54802, 54802, 54802, 54802, 1295, 54802, 54802, 950, + -3054, -3054, -3054, 21945, 14, 14, 1841, 13105, 121, -3054, + 20385, 20385, 206, 248, 64687, 1801, -3054, -3054, 828, 1848, + 160, -3054, 1673, 54802, 54802, 54802, 54802, 64687, 54802, 1491, + -3054, -3054, -3054, -3054, -3054, 1624, -3054, 1626, 1990, 2616, + -3054, 1991, 50842, 166, 3478, 1997, 1674, 1999, 13625, 2097, + 2000, 2144, 1640, 1879, -3054, -3054, 1867, -3054, 64687, 2157, + -3054, 122, -3054, 47377, -3054, 228, -3054, 1873, 239, -3054, + 15185, 20385, -3054, -3054, -3054, -3054, -3054, -3054, 1292, 28152, + -3054, 832, -3054, -3054, 2121, 950, 2121, 780, -3054, -3054, + 2121, -3054, 2106, 2121, -3054, -3054, 69637, -3054, 8220, -3054, + 20385, 20385, -3054, 20385, 1996, -3054, 2158, 2158, 69637, 25065, + 25065, 25065, 25065, 25065, 25065, 511, 1327, 25065, 25065, 25065, + 25065, 25065, 25065, 25065, 25065, 25065, 26625, 268, -3054, -3054, + 838, 2128, 20385, 20385, 2004, 1996, 20385, -3054, 69637, 1657, + -3054, 1658, 1662, 20385, -3054, 69637, -3054, 54802, 1663, 31, + 27, -3054, 1665, 1666, -3054, 1597, -3054, 856, 935, 54802, + 3395, 5093, 5529, -3054, -3054, 20385, 2001, 1896, 31638, -3054, + 20385, 1669, -3054, -3054, 31638, 2018, -3054, -3054, -3054, 36094, + 1896, 69637, 849, -3054, 54802, 69637, 860, 20385, -3054, 15185, + 2183, 69637, 2148, 64687, 64687, 2186, 1680, 1682, 2017, 1768, + -3054, 1769, 1770, 1771, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, 69637, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, 1675, 1687, 20385, 20385, 85, -3054, 8294, 1686, + 1690, 6447, -3054, 1685, -3054, 1691, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, 1693, -3054, 1700, -3054, 1704, 1716, 1719, + 1728, 1733, 54802, -3054, 22465, -3054, 64687, -3054, -3054, 20385, + 20385, 54802, -3054, 1697, -3054, 1734, 1735, 8386, -3054, -3054, + -3054, 262, 1073, 45260, 295, 45316, 45316, 45316, 45363, -3054, + -3054, -3054, 1726, -3054, 25065, 25065, -3054, 3089, 1200, 10524, + -3054, -3054, 2047, -3054, 1065, -3054, 1717, -3054, -3054, 2724, + -3054, 39561, 7629, 20385, 180, -3054, 20385, 1545, 20385, 1802, + 45316, 45316, 45316, 246, 246, 262, 262, 262, 1073, 295, + -3054, -3054, -3054, 1730, 20385, 47377, -3054, 1737, 1738, 2098, + 1329, 20385, -3054, -3054, 31638, 1512, -8, 1512, 2017, 7141, + -3054, 1051, -3054, -3054, 1051, 44932, 54802, -3054, -3054, 2006, + 1742, 31638, 1785, 2220, 2202, 64687, -3054, -3054, 1744, 1901, + 1760, -3054, -3054, 1765, 20385, 2073, 1765, -3054, 1869, -10, + 1979, 1062, 1062, 656, 1982, -3054, -3054, 1827, -3054, -3054, + -3054, 20385, 14145, 1394, -3054, 1401, -3054, -3054, -3054, -3054, + -3054, 1755, -3054, 2031, -3054, 54802, -3054, -3054, 25065, 2216, + 20385, 36589, 2222, 2019, -3054, -3054, -3054, 1854, 1372, 20385, + 2009, -3054, 156, 1778, 2139, 357, 2091, 64687, -3054, 350, + 459, -3054, 817, 2145, 239, 2147, 239, 47377, 47377, 47377, + 879, -3054, -3054, -3054, 950, -3054, 316, 881, -3054, -3054, + -3054, -3054, 1868, 644, 2616, 1372, -3054, -3054, -3054, -3054, + -3054, -3054, -3054, 184, 690, 1372, 1874, -3054, 1877, -3054, + 1878, 704, 1372, -3054, -3054, 1639, 8960, 44932, 408, 121, + 121, 121, 15185, -3054, 2032, 2033, 1797, 44932, 44932, 153, + -3054, -3054, -3054, -3054, 1786, -3054, 170, -3054, 64687, -3054, + -3054, -3054, 1801, 1831, 54802, 2616, 1803, 2275, 1222, 1435, + 1756, -3054, 1964, 29, 1551, -3054, 64687, 54802, 54802, 54802, + 63712, -3054, -3054, -3054, 1804, 1800, -3054, 20, 2035, 2034, + 54802, 1844, 54802, 1375, 2285, 54802, -3054, 908, 17785, 64687, + -3054, 47377, 2175, 54802, 1789, -3054, -3054, -3054, -3054, 64687, + -3054, -3054, 44932, -3054, -3054, 48367, -3054, -3054, -3054, -3054, + -3054, 47377, -3054, 950, -3054, 950, 2049, 64687, 42254, 950, + 42749, 950, 1814, -3054, 44932, 40064, 44932, 2004, -3054, 155, + 2158, 527, 527, 527, 4897, 2159, 232, 1817, 527, 527, + 527, 146, 146, 155, 155, 155, 2158, 268, 868, 49357, + 1818, -3054, 44932, 44932, -3054, -3054, 1819, -3054, -3054, -3054, + -3054, 1820, 1821, -3054, -3054, -3054, -3054, -3054, -3054, 64687, + 1086, 1512, 417, 417, 417, 417, -3054, 54802, 54802, 54802, + 44932, 2278, 2154, -3054, 1896, 44932, 54802, -3054, 29157, 54802, + -3054, 2174, -3054, 2263, -3054, 54802, 910, -3054, -3054, -3054, + 919, 1834, 1682, 69637, 953, 954, -3054, 2017, 147, 1843, + 1510, 1113, 582, 1402, -3054, -3054, -3054, 1845, 44636, 20385, + -3054, 2204, -3054, -3054, -3054, 20385, 20385, -3054, 39561, -3054, + -3054, -3054, -3054, 372, 372, 1846, 10524, 44858, -3054, 2150, + 40103, 44932, -3054, 1698, -3054, -3054, 7141, 20385, 2351, 3644, + 20385, 1850, 20385, 2184, -3054, -3054, 1849, -3054, -3054, 69637, + 20385, 1860, 4152, 25065, 25065, 4537, -3054, 4675, 20385, 10524, + -3054, 41178, 1852, 1859, 1841, 18305, -3054, 2078, 1861, -3054, + 2001, 121, 2001, 1866, -3054, -3054, -3054, -3054, 5103, -3054, + 20385, 2026, 64687, 371, 1974, 995, -3054, 638, 38074, 1785, + 20385, 244, -3054, -3054, 1880, -3054, 1765, -3054, -3054, -3054, + 2101, -3054, -3054, -3054, 54802, -3054, 1884, -3054, 35599, 2213, + 11044, -3054, 35599, 54802, 54802, 40142, 2248, -3054, 64687, 64687, + 64687, -3054, 64687, 1883, 1887, 850, 1890, 391, -3054, 3681, + 850, 2228, 579, 1375, 259, 2963, 431, -3054, -3054, -3054, + 1966, 54802, -3054, 64687, -3054, -3054, -3054, -3054, -3054, 47872, + -3054, -3054, 39065, 47377, -3054, 47377, 54802, 54802, 54802, 54802, + 54802, 54802, 54802, 54802, 54802, 54802, 1292, 20385, -3054, 20385, + 1893, 1897, 1898, 1841, -3054, -3054, -3054, 237, -3054, 1904, + -3054, -3054, -3054, -51, -3054, 170, 1903, -3054, 50842, 2206, + 1674, 2375, 1910, 1831, 703, 64192, -3054, 1913, 1911, 1008, + 2616, 1916, 2376, -3054, 166, 50842, -3054, -3054, -3054, 2332, + -3054, 682, 220, -3054, 1222, -3054, 2206, 1435, -3054, 2206, + 44932, 1756, 1009, 64687, 1977, -3054, 239, 1013, -3054, -3054, + -3054, -3054, -3054, 64687, 1914, -3054, 1914, -3054, -3054, 1914, + -3054, -3054, -3054, -3054, 25065, 2264, 1924, 69637, -3054, -3054, + 54802, -3054, -3054, -3054, 1014, 1925, 2001, 54802, 54802, 54802, + 54802, -3054, -3054, -3054, 18825, 20385, 1967, -3054, 1930, 12084, + 2241, -3054, 28658, -3054, 214, 1931, 36094, 64687, -3054, -3054, + -3054, -3054, 2017, -3054, -3054, 64687, -3054, 1934, -3054, 1938, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, 20385, + 44932, -3054, 44932, -3054, -3054, -3054, -3054, -3054, 7702, -3054, + 1935, 1939, 64687, 20385, -3054, -3054, -3054, 392, 20385, 20385, + 3089, -3054, 45403, 20385, 69637, 1015, 3089, 354, 20385, 3898, + 4474, 20385, 20385, 5451, 40189, -3054, 22985, 14665, -3054, 1941, + 20385, 40228, 38569, -3054, 31638, 2154, 1948, 2154, 950, 1949, + 44932, 20385, -3054, -3054, -3054, -3054, 1989, 381, 33123, 2176, + -3054, 1963, 64687, -3054, 2026, 44932, -3054, -3054, 39561, -3054, + -3054, -3054, -3054, -3054, 2408, 82, 1953, 1955, -3054, 1328, + -3054, -3054, 64687, 1957, -3054, 1960, 850, -3054, 64687, 2002, + -3054, 258, 2272, 104, -3054, 20385, -3054, 2360, 2439, 3681, + 1968, 64687, 54802, 25065, -3054, 297, 202, -3054, 2255, 54802, + 2002, 2396, -3054, -3054, -3054, 391, -3054, 2295, 2208, -3054, + 231, -3054, 20385, 391, 2210, 238, 64687, -3054, -3054, 2709, + -3054, 69637, 239, 239, -3054, 1500, 1978, 1983, 1987, 1988, + 1994, 1998, 2011, 2015, 2021, 2022, -3054, 2023, 2030, 2038, + 2042, 2044, 2052, 2057, 2061, 1517, 2068, -3054, 2070, 1880, + 2071, 2074, 2077, 2083, 2084, 70597, 2085, 2086, 2087, 2088, + 1525, 2089, 425, 985, -3054, -3054, -3054, -3054, 2092, -3054, + -3054, -3054, 1254, 2003, -3054, -3054, -3054, 2079, -3054, 2090, + -3054, -3054, -3054, -3054, -3054, -3054, 1992, 2076, -3054, -3054, + -3054, 121, 2020, 2050, 64687, 1292, 169, 64687, 2094, 1844, + 2458, 19345, 47377, 764, 2238, 2099, -3054, 950, 1674, -3054, + 50842, 3843, 490, 2034, -3054, 277, 1844, -3054, 2406, 1674, + 2039, 2476, -3054, 2100, -3054, 2245, 64687, 2102, -3054, -3054, + 48367, 1914, 5145, 25065, 69637, 1021, 1037, -3054, 2508, 2240, + 2154, -3054, -3054, -3054, -3054, -3054, 2104, 16, 2115, 10004, + 2093, -3054, -3054, -3054, -3054, -3054, -3054, 44932, 44932, 64687, + 2280, -3054, -3054, 2112, 2105, 37084, 2572, 2117, -3054, 15185, + -3054, 2435, -3054, 30151, -3054, 1682, 2122, 1682, 69637, 1682, + -3054, -3054, 44932, 20385, -3054, -3054, 41217, 2443, 3089, 3089, + 45403, 1060, -3054, 3089, 20385, 20385, 3089, 3089, 20385, -3054, + 9482, 503, -3054, 1066, -3054, 40578, -3054, 71077, -3054, -3054, + 1967, 950, 1967, -3054, -3054, 2118, -3054, -3054, -3054, 2178, + -3054, -3054, 1107, 2547, 2026, 20385, -3054, -3054, 2124, 35599, + -3054, -3054, -3054, -3054, 35599, 850, -3054, 2296, 2002, 2127, + -3054, -3054, -3054, -3054, -3054, -3054, 40664, -3054, 66, 20385, + -3054, 1235, 4897, -3054, -3054, -3054, -3054, 2002, 1222, -3054, + 54802, 2603, 2494, -3054, -3054, 44932, -3054, -3054, 2017, 2017, + -3054, -3054, 2263, -3054, -3054, 2132, -3054, -3054, 1254, -63, + 39065, 54802, 54802, -3054, -3054, 2135, -3054, -3054, -3054, -3054, + -3054, -51, 2534, 1123, 166, -3054, 2206, 2206, 44932, 1148, + 54802, 2507, 50842, -3054, 2620, 2146, 54802, 1844, 1071, 1071, + -3054, 2297, -3054, 2298, -3054, -3054, 2622, 315, -3054, 19865, + 47377, 54802, -3054, -3054, 32628, -3054, 5145, 1152, -3054, -3054, + 2152, 2155, -3054, 1967, 20385, 2160, 20385, -3054, 23505, 2628, + 2162, -3054, 20385, 2218, 27145, -3054, 20385, -3054, 54802, 59752, + 14, 2163, 59752, -3054, -3054, -3054, -3054, -3054, 20385, -3054, + 3089, 3089, 3089, 20385, -3054, 20385, -3054, -3054, -3054, 2364, + 2280, -3054, 2280, 20385, 2206, 638, 1892, 64687, 15, -3054, + 44932, -3054, -3054, -3054, 54802, -3054, 47377, -3054, 850, -9, + 2164, 20385, 40703, 2402, -3054, -3054, 2436, -3054, 2493, -3054, + 2231, 530, 2247, -3054, -3054, -3054, -3054, 1292, 950, 1674, + 2034, 2039, -3054, -3054, 2179, 54802, 2206, 166, 682, -3054, + -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, -3054, + -3054, -3054, 2206, 2614, 2397, 2618, 2206, 44932, 1157, 1977, + 20385, 83, -3054, 1158, 2615, -3054, -3054, 2685, 2280, 2181, + 23505, 2182, -3054, 2189, 64687, 44932, 2330, -3054, -3054, 2194, + -3054, -3054, 121, 20385, -3054, 41267, 2191, 2200, 2655, 1841, + 2218, 2218, -3054, 381, -3054, -3054, 2623, 32628, 2584, 1222, + 850, 2215, 1168, -3054, -3054, -3054, -3054, -3054, 2616, -3054, + 40742, 2449, 223, 2437, 2164, 20385, -3054, 2286, -3054, -3054, + -3054, 2684, -3054, -3054, 50842, 2214, 2039, 2034, 1844, 2039, + 2438, -3054, 2441, -3054, -3054, 2217, 40828, 64687, 64687, 1674, + 32628, 64687, 2212, 2218, -3054, 2219, -3054, -3054, -3054, 53812, + -3054, 2221, 2223, -3054, -3054, -3054, 20385, 187, -3054, -3054, + 2270, 54802, 1177, 43, 2436, 39065, -3054, 47377, 2451, -9, + 2533, -3054, -3054, -3054, -3054, 149, 2452, -3054, 2454, -3054, + 44932, -3054, 2206, 50842, -3054, -3054, -3054, -3054, -3054, -3054, + 32628, 2615, -3054, 367, -3054, 1512, -3054, 367, -3054, -3054, + -3054, -3054, 950, -3054, 1461, 24025, 24025, 24025, 2229, 2206, + -3054, 1512, -3054, 2361, 2437, -3054, -3054, -3054, -3054, -3054, + 165, 165, 2624, -3054, 2299, -3054, 2039, 1179, 64687, 1765, + -3054, 1765, 25585, 2386, 222, 43814, 2604, -3054, 2604, 2604, + -3054, -3054, -3054, 38074, -3054, -3054, 2728, -3054, 217, -3054, + -3054, -3054, 1674, 367, -3054, -3054, 2721, -3054, -3054, -3054, + -3054, -3054, 196, -3054, -3054, -3054, 1512, 850, -3054, -3054, + -3054, 1512, 1765, 24545, 2393, -3054, 2462, -3054, -3054, -3054, + -3054, -3054, -3054, -3054 }; -# endif -/* YYR1[YYN] -- Symbol number of symbol that rule YYN derives. */ -static const yytype_uint16 yyr1[] = +/* YYDEFACT[STATE-NUM] -- Default reduction number in state STATE-NUM. + Performed when YYTABLE does not specify something else to do. Zero + means the default is an error. */ +static const yytype_int16 yydefact[] = { - 0, 522, 523, 524, 524, 525, 525, 525, 525, 525, - 525, 525, 525, 525, 525, 525, 525, 525, 525, 525, - 525, 525, 525, 525, 525, 525, 525, 525, 525, 525, - 525, 525, 525, 525, 525, 525, 525, 525, 525, 525, - 525, 525, 525, 526, 526, 526, 526, 526, 526, 526, - 526, 527, 527, 528, 528, 529, 529, 529, 529, 530, - 530, 531, 531, 531, 531, 531, 531, 531, 531, 531, - 531, 531, 531, 531, 531, 531, 531, 531, 531, 531, - 531, 531, 531, 531, 531, 531, 531, 531, 531, 531, - 532, 532, 533, 533, 533, 533, 534, 534, 535, 536, - 536, 536, 537, 537, 537, 537, 538, 538, 539, 539, - 539, 540, 540, 541, 542, 542, 543, 544, 545, 545, - 545, 545, 546, 546, 546, 546, 546, 546, 546, 546, - 546, 546, 546, 546, 546, 547, 547, 548, 549, 549, - 549, 549, 549, 550, 550, 551, 551, 551, 552, 552, - 552, 553, 553, 554, 555, 555, 556, 556, 556, 557, - 557, 557, 558, 558, 558, 559, 559, 560, 560, 561, - 561, 562, 562, 563, 563, 564, 564, 565, 565, 566, - 566, 567, 567, 568, 569, 569, 569, 570, 570, 571, - 571, 572, 572, 572, 573, 573, 573, 574, 574, 575, - 575, 575, 576, 576, 577, 577, 578, 578, 579, 579, - 580, 580, 581, 581, 581, 582, 582, 582, 582, 583, - 583, 583, 583, 583, 583, 583, 583, 583, 583, 583, - 583, 583, 583, 584, 584, 585, 585, 585, 586, 586, - 586, 586, 586, 586, 587, 587, 587, 588, 589, 589, - 589, 590, 590, 591, 591, 591, 591, 591, 591, 592, - 592, 593, 594, 594, 594, 594, 594, 595, 595, 595, - 595, 596, 596, 596, 596, 596, 596, 596, 596, 597, - 597, 598, 598, 599, 599, 599, 600, 601, 602, 602, - 602, 602, 602, 603, 603, 603, 603, 604, 605, 605, - 606, 606, 607, 607, 607, 607, 607, 607, 607, 607, - 608, 608, 609, 610, 610, 610, 610, 611, 611, 611, - 611, 612, 613, 613, 613, 614, 615, 615, 615, 615, - 615, 615, 616, 616, 617, 617, 618, 619, 619, 619, - 620, 620, 621, 621, 622, 622, 622, 623, 624, 624, - 625, 625, 626, 627, 627, 627, 627, 628, 628, 629, - 629, 630, 630, 630, 631, 631, 631, 631, 631, 631, - 632, 632, 633, 633, 633, 633, 634, 635, 635, 635, - 635, 635, 635, 635, 635, 636, 636, 637, 637, 637, - 637, 637, 637, 638, 638, 638, 638, 638, 638, 638, - 638, 638, 638, 638, 638, 638, 638, 638, 638, 638, - 638, 639, 639, 639, 639, 639, 639, 640, 640, 641, - 641, 641, 642, 642, 642, 643, 643, 643, 643, 643, - 643, 644, 644, 645, 645, 646, 647, 647, 647, 648, - 648, 648, 649, 649, 650, 650, 651, 651, 652, 652, - 653, 653, 654, 654, 655, 655, 656, 656, 656, 656, - 656, 656, 657, 658, 658, 659, 659, 660, 660, 661, - 661, 661, 661, 661, 661, 661, 661, 661, 661, 661, - 661, 661, 661, 661, 661, 662, 663, 663, 663, 664, - 664, 665, 665, 666, 666, 667, 667, 667, 667, 667, - 667, 667, 667, 668, 668, 669, 669, 670, 670, 670, - 670, 670, 670, 670, 670, 670, 670, 670, 670, 670, - 670, 670, 670, 670, 670, 670, 671, 671, 672, 672, - 673, 673, 674, 674, 674, 675, 675, 676, 676, 677, - 677, 677, 678, 678, 679, 680, 680, 680, 681, 681, - 682, 682, 682, 682, 682, 682, 682, 682, 682, 683, - 683, 684, 684, 684, 685, 686, 686, 687, 687, 688, - 688, 688, 689, 689, 690, 690, 691, 691, 692, 692, - 693, 693, 693, 694, 694, 694, 695, 695, 695, 695, - 696, 696, 697, 697, 697, 697, 698, 698, 699, 699, - 699, 699, 699, 699, 700, 700, 701, 701, 702, 702, - 702, 702, 703, 704, 704, 705, 705, 706, 706, 706, - 706, 706, 707, 708, 708, 708, 709, 709, 710, 710, - 711, 711, 712, 712, 712, 713, 713, 714, 714, 715, - 715, 715, 715, 715, 716, 717, 718, 719, 720, 720, - 721, 721, 722, 722, 723, 723, 724, 724, 725, 725, - 726, 727, 727, 727, 727, 728, 728, 729, 729, 729, - 730, 730, 731, 731, 732, 732, 733, 733, 734, 734, - 735, 735, 735, 735, 735, 735, 735, 735, 735, 735, - 736, 736, 737, 737, 737, 738, 738, 739, 739, 740, - 740, 741, 741, 742, 742, 743, 744, 744, 745, 745, - 745, 745, 745, 745, 745, 745, 745, 745, 745, 746, - 746, 746, 746, 747, 747, 748, 748, 748, 748, 748, - 749, 749, 749, 749, 749, 749, 750, 750, 751, 751, - 752, 752, 752, 752, 753, 753, 754, 755, 755, 756, - 756, 757, 757, 758, 758, 759, 759, 760, 761, 761, - 762, 762, 763, 763, 764, 764, 765, 765, 765, 765, - 765, 765, 765, 765, 765, 765, 766, 766, 767, 767, - 767, 768, 768, 768, 768, 768, 768, 768, 769, 769, - 769, 769, 770, 771, 771, 772, 772, 772, 772, 772, - 772, 772, 772, 772, 772, 772, 773, 773, 774, 774, - 775, 775, 776, 777, 778, 778, 779, 779, 780, 781, - 782, 782, 782, 782, 782, 782, 783, 783, 784, 784, - 784, 784, 785, 786, 786, 786, 787, 787, 788, 788, - 789, 789, 790, 790, 791, 791, 792, 792, 793, 793, - 794, 794, 795, 795, 796, 796, 797, 797, 798, 798, - 799, 799, 799, 799, 799, 799, 799, 799, 799, 799, - 799, 799, 799, 799, 799, 799, 799, 799, 799, 799, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 801, 801, 801, 801, 801, 801, 801, 801, 801, 801, - 801, 801, 801, 801, 801, 801, 801, 801, 801, 801, - 801, 801, 801, 801, 801, 802, 802, 802, 803, 803, - 803, 803, 803, 803, 803, 803, 803, 803, 803, 803, - 803, 804, 804, 804, 804, 804, 804, 805, 806, 806, - 806, 806, 806, 806, 807, 807, 808, 808, 809, 809, - 809, 809, 809, 809, 809, 809, 809, 809, 809, 809, - 809, 809, 810, 810, 811, 811, 812, 812, 812, 813, - 813, 814, 814, 815, 815, 816, 817, 817, 817, 818, - 819, 819, 820, 820, 821, 821, 821, 821, 822, 822, - 823, 823, 823, 823, 823, 824, 824, 824, 824, 824, - 825, 825, 826, 826, 827, 828, 828, 829, 829, 830, - 831, 831, 832, 832, 833, 833, 834, 834, 834, 835, - 835, 836, 836, 836, 836, 836, 836, 836, 836, 836, - 836, 836, 836, 836, 836, 837, 837, 838, 838, 839, - 839, 839, 839, 839, 839, 839, 839, 840, 840, 841, - 841, 842, 842, 843, 843, 844, 844, 845, 845, 846, - 846, 847, 847, 847, 848, 848, 849, 849, 850, 850, - 850, 850, 850, 850, 850, 850, 850, 850, 850, 850, - 850, 850, 851, 851, 852, 853, 853, 854, 854, 854, - 854, 854, 854, 855, 856, 857, 857, 857, 858, 858, - 859, 860, 860, 861, 862, 862, 863, 863, 864, 864, - 543, 543, 543, 543, 865, 865, 866, 866, 867, 867, - 867, 868, 868, 868, 868, 868, 869, 869, 870, 870, - 871, 871, 872, 872, 873, 873, 874, 874, 874, 875, - 875, 876, 876, 877, 878, 878, 879, 879, 880, 880, - 880, 881, 881, 882, 882, 883, 883, 884, 884, 885, - 886, 886, 887, 887, 887, 887, 887, 887, 887, 887, - 887, 887, 887, 887, 887, 887, 888, 889, 889, 889, - 890, 890, 890, 891, 891, 891, 892, 892, 893, 893, - 894, 894, 895, 896, 896, 897, 898, 898, 899, 899, - 899, 899, 899, 900, 900, 900, 901, 901, 902, 902, - 902, 902, 903, 903, 904, 905, 905, 906, 906, 907, - 907, 908, 908, 909, 909, 910, 910, 910, 910, 910, - 910, 911, 911, 912, 912, 913, 913, 914, 915, 915, - 915, 915, 916, 916, 917, 917, 917, 918, 918, 918, - 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, - 918, 918, 918, 918, 918, 918, 918, 918, 918, 918, - 918, 918, 918, 918, 918, 918, 918, 919, 919, 919, - 920, 920, 921, 921, 922, 922, 923, 923, 923, 923, - 924, 925, 925, 926, 926, 926, 926, 927, 927, 927, - 927, 928, 928, 929, 930, 930, 930, 930, 930, 930, - 930, 931, 931, 932, 932, 932, 932, 932, 933, 933, - 934, 934, 935, 935, 935, 935, 935, 936, 936, 936, - 936, 936, 937, 937, 938, 938, 939, 939, 940, 940, - 941, 941, 941, 942, 942, 943, 943, 944, 944, 945, - 945, 946, 946, 947, 947, 948, 948, 949, 949, 949, - 949, 950, 950, 951, 951, 951, 952, 952, 952, 952, - 952, 952, 952, 952, 953, 953, 954, 954, 955, 955, - 956, 956, 957, 958, 958, 958, 958, 958, 959, 959, - 959, 959, 960, 960, 960, 961, 961, 961, 962, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 963, - 963, 963, 963, 963, 963, 963, 963, 963, 963, 964, - 964, 964, 964, 964, 964, 964, 964, 964, 964, 964, - 964, 964, 964, 964, 964, 964, 964, 964, 964, 964, - 964, 964, 964, 964, 964, 964, 964, 964, 964, 964, - 964, 964, 964, 964, 964, 964, 964, 964, 964, 964, - 964, 964, 964, 964, 964, 964, 964, 964, 964, 964, - 964, 964, 964, 965, 965, 965, 965, 965, 965, 965, - 965, 965, 965, 965, 965, 965, 965, 965, 965, 965, - 965, 965, 965, 965, 965, 965, 965, 965, 965, 965, - 965, 966, 966, 966, 966, 966, 966, 966, 966, 966, - 966, 966, 966, 966, 966, 966, 966, 966, 966, 966, - 966, 966, 966, 966, 966, 966, 966, 966, 966, 966, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 967, 967, 967, 967, 967, 967, 967, 967, 967, - 967, 968, 968, 968, 968, 968, 968, 968, 968, 968, - 968, 968, 968, 968, 968, 968, 968, 968, 968, 968, - 968, 968, 968, 968, 968, 968, 968, 968, 968, 968, - 968, 968, 968, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969, 969, 969, 969, 969, 969, - 969, 969, 969, 969, 969 + 155, 263, 0, 1380, 1379, 1450, 263, 0, 1315, 0, + 263, 485, 401, 0, 1471, 1470, 0, 207, 263, 0, + 155, 0, 0, 0, 0, 0, 0, 548, 551, 549, + 0, 0, 0, 263, 588, 0, 1472, 263, 0, 0, + 580, 550, 0, 1428, 0, 0, 0, 0, 0, 2, + 4, 5, 6, 7, 8, 9, 23, 10, 11, 12, + 13, 19, 14, 15, 16, 17, 18, 20, 21, 22, + 25, 24, 26, 27, 1338, 28, 29, 30, 31, 0, + 32, 33, 34, 35, 36, 524, 510, 593, 523, 0, + 0, 154, 692, 531, 37, 38, 39, 40, 41, 42, + 43, 1473, 1469, 44, 262, 261, 255, 0, 0, 0, + 0, 0, 1449, 0, 0, 256, 111, 1497, 1498, 1499, + 1500, 1501, 1502, 1503, 1504, 1505, 1506, 1507, 1871, 1508, + 1509, 1510, 1511, 1512, 1872, 1513, 1514, 1515, 1817, 1818, + 1873, 1819, 1820, 1516, 1517, 1518, 1519, 1520, 1521, 1522, + 1523, 1524, 1525, 1821, 1822, 1526, 1527, 1528, 1529, 1530, + 1823, 1874, 1824, 1531, 1532, 1533, 1534, 1535, 1875, 1536, + 1537, 1538, 1539, 1540, 1541, 1542, 1543, 1544, 1876, 1545, + 1546, 1547, 1548, 1549, 1550, 1551, 1552, 1553, 1554, 1825, + 1555, 1556, 1826, 1557, 1558, 1559, 1560, 1561, 1562, 1563, + 1564, 1565, 1566, 1567, 1568, 1569, 1570, 1571, 1572, 1573, + 1574, 1575, 1576, 1577, 1578, 1579, 1580, 1581, 1582, 1583, + 1827, 1584, 1585, 1586, 1587, 1588, 1828, 1589, 1590, 1591, + 1829, 1592, 1593, 1594, 1877, 1878, 1595, 1596, 1830, 1880, + 1597, 1598, 1831, 1832, 1599, 1600, 1601, 1602, 1603, 1604, + 1605, 1606, 1607, 1881, 1608, 1609, 1610, 1611, 1612, 1613, + 1614, 1615, 1616, 1617, 1618, 1619, 1882, 1833, 1620, 1621, + 1622, 1623, 1624, 1834, 1835, 1836, 1625, 1883, 1884, 1626, + 1885, 1627, 1628, 1629, 1630, 1631, 1632, 1633, 1886, 1634, + 1887, 1635, 1636, 1637, 1638, 1639, 1640, 1641, 1642, 1837, + 1643, 1644, 1645, 1646, 1647, 1648, 1649, 1650, 1651, 1652, + 1653, 1654, 1655, 1656, 1657, 1658, 1659, 1660, 1661, 1662, + 1838, 1889, 1839, 1663, 1664, 1665, 1840, 1666, 1667, 1890, + 1668, 1841, 1669, 1842, 1670, 1671, 1672, 1673, 1674, 1675, + 1676, 1677, 1678, 1679, 1843, 1891, 1680, 1892, 1844, 1681, + 1682, 1683, 1684, 1685, 1686, 1687, 1688, 1689, 1690, 1691, + 1692, 1693, 1845, 1893, 1694, 1695, 1846, 1696, 1697, 1698, + 1699, 1700, 1701, 1702, 1703, 1704, 1705, 1706, 1707, 1847, + 1708, 1709, 1710, 1711, 1712, 1713, 1714, 1715, 1716, 1717, + 1718, 1719, 1720, 1721, 1722, 1723, 1724, 1725, 1726, 1894, + 1727, 1728, 1729, 1848, 1730, 1731, 1732, 1733, 1734, 1735, + 1736, 1737, 1738, 1739, 1740, 1741, 1742, 1743, 1744, 1745, + 1746, 1747, 1748, 1849, 1749, 1750, 1895, 1751, 1752, 1850, + 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1760, 1761, 1762, + 1763, 1764, 1765, 1851, 1766, 1852, 1767, 1768, 1769, 1897, + 1770, 1771, 1772, 1773, 1774, 1775, 1853, 1854, 1776, 1777, + 1855, 1778, 1856, 1779, 1780, 1857, 1781, 1782, 1783, 1784, + 1785, 1786, 1787, 1788, 1789, 1790, 1791, 1792, 1793, 1794, + 1795, 1796, 1797, 1858, 1859, 1798, 1898, 1799, 1800, 1801, + 1802, 1803, 1804, 1805, 1806, 1807, 1808, 1809, 1810, 1811, + 1812, 1860, 1861, 1862, 1863, 1864, 1865, 1866, 1867, 1868, + 1869, 1870, 1813, 1814, 1815, 1816, 0, 112, 113, 1262, + 1480, 0, 1240, 111, 1830, 1837, 1851, 1313, 1314, 112, + 0, 258, 484, 0, 0, 0, 0, 0, 0, 209, + 0, 395, 394, 0, 1304, 400, 0, 0, 0, 115, + 107, 1696, 114, 1239, 105, 121, 2041, 2042, 2043, 2044, + 1928, 2045, 2046, 2047, 2048, 1929, 2049, 1930, 1931, 1932, + 1933, 1934, 1935, 2050, 2051, 2052, 1937, 1936, 2053, 1938, + 2054, 1939, 2055, 1940, 1941, 2056, 2057, 1942, 1551, 1943, + 1944, 2058, 2059, 2060, 2061, 2062, 2063, 2064, 2065, 2066, + 1945, 1946, 2067, 2068, 1947, 2069, 2070, 1948, 2071, 1949, + 1950, 1951, 2072, 2073, 1952, 1953, 2074, 1954, 2075, 2076, + 1955, 1956, 1959, 1957, 2077, 1958, 2078, 1960, 1961, 1962, + 2079, 2080, 1963, 1964, 2081, 1965, 1966, 1967, 1968, 1969, + 2082, 1970, 2083, 1971, 1972, 2084, 2085, 2086, 2087, 2088, + 1974, 1973, 1975, 1976, 2089, 2090, 2091, 2092, 1977, 1978, + 1979, 2093, 2094, 1980, 2095, 2096, 1981, 1982, 2097, 1983, + 1984, 2098, 1985, 1986, 2099, 1987, 1988, 2100, 2101, 2102, + 1989, 2103, 1990, 1991, 2104, 2105, 1992, 1993, 2106, 1994, + 2107, 2108, 2109, 2110, 1995, 1996, 2111, 1997, 2112, 2113, + 2114, 2115, 1998, 1999, 2000, 2001, 2002, 2003, 2004, 2005, + 2006, 2007, 2008, 1446, 123, 122, 124, 0, 419, 420, + 0, 430, 0, 412, 417, 413, 0, 439, 432, 440, + 421, 411, 433, 422, 410, 208, 0, 441, 427, 415, + 0, 0, 0, 0, 259, 220, 401, 0, 155, 0, + 1342, 1343, 1344, 1345, 1346, 1347, 1352, 1348, 1349, 1350, + 1351, 1353, 1354, 1355, 1356, 1357, 1333, 1338, 1358, 1359, + 1360, 1361, 1362, 1363, 1364, 1365, 1366, 1367, 1368, 1369, + 1370, 1371, 0, 1315, 0, 0, 1823, 1874, 1828, 0, + 1841, 0, 1844, 1845, 1730, 1852, 1855, 1856, 1857, 1858, + 0, 760, 114, 109, 744, 0, 526, 696, 706, 744, + 749, 1026, 772, 1027, 0, 116, 1414, 1413, 1409, 1408, + 194, 1277, 1458, 1597, 1637, 1747, 1853, 1776, 1476, 1459, + 1453, 1457, 260, 587, 585, 0, 1211, 1597, 1637, 1734, + 1747, 1853, 1388, 1392, 0, 257, 1478, 1463, 0, 1464, + 114, 532, 579, 0, 264, 1427, 0, 1432, 0, 1710, + 559, 562, 1271, 560, 524, 0, 0, 1, 155, 1337, + 1442, 0, 161, 0, 583, 583, 0, 583, 0, 516, + 0, 0, 524, 519, 523, 693, 1475, 1853, 1776, 1462, + 1465, 1606, 0, 0, 1606, 0, 1606, 0, 1606, 0, + 0, 1452, 1195, 0, 1241, 117, 0, 0, 1326, 1322, + 1327, 1323, 1328, 1321, 1320, 1329, 1325, 0, 0, 0, + 366, 399, 398, 397, 396, 401, 1606, 1288, 0, 448, + 449, 0, 0, 0, 0, 0, 205, 1299, 108, 106, + 1606, 1447, 428, 429, 0, 418, 414, 416, 0, 0, + 1606, 1266, 438, 434, 1606, 438, 1233, 1606, 0, 0, + 212, 0, 394, 1335, 1372, 1995, 1373, 1374, 1386, 0, + 1387, 1377, 1341, 155, 0, 483, 1312, 1410, 0, 0, + 0, 1147, 744, 749, 0, 0, 762, 0, 1166, 0, + 1172, 0, 0, 0, 744, 531, 0, 706, 761, 110, + 0, 742, 743, 634, 634, 588, 0, 569, 754, 0, + 0, 757, 755, 0, 757, 0, 0, 0, 757, 753, + 714, 0, 634, 0, 742, 745, 634, 0, 764, 1332, + 0, 0, 0, 0, 0, 1456, 1454, 1455, 1460, 0, + 0, 0, 1243, 1245, 1246, 1115, 1256, 1012, 0, 1818, + 1819, 1820, 1187, 1821, 1822, 1824, 1825, 1826, 966, 1571, + 1827, 1254, 1829, 1831, 1832, 1834, 1835, 1836, 1837, 1838, + 1839, 0, 1255, 1842, 1675, 1847, 1848, 1850, 1853, 1854, + 1253, 1859, 0, 0, 0, 1222, 1138, 0, 0, 0, + 1011, 0, 1188, 1196, 1007, 0, 0, 808, 809, 830, + 831, 810, 836, 837, 839, 811, 0, 1218, 900, 995, + 1206, 1014, 1045, 1016, 1025, 1003, 1082, 996, 0, 1006, + 998, 1214, 569, 1212, 0, 999, 1242, 569, 1210, 1391, + 1389, 1395, 1390, 0, 0, 0, 0, 0, 110, 1434, + 1435, 1426, 1424, 1425, 1423, 1422, 1429, 0, 1431, 1338, + 1133, 1135, 0, 561, 0, 0, 0, 513, 512, 514, + 3, 1271, 0, 0, 0, 0, 581, 582, 0, 0, + 0, 0, 0, 0, 0, 0, 677, 608, 609, 611, + 674, 678, 686, 0, 0, 0, 0, 0, 520, 0, + 1474, 1468, 1466, 0, 0, 0, 139, 139, 0, 0, + 0, 0, 0, 99, 48, 92, 0, 0, 0, 0, + 234, 247, 0, 0, 0, 0, 0, 244, 0, 0, + 227, 221, 223, 50, 0, 139, 0, 46, 0, 0, + 0, 52, 1450, 0, 483, 1194, 0, 119, 120, 118, + 111, 0, 2009, 1871, 1872, 1873, 1874, 1824, 1875, 1876, + 0, 1877, 1878, 1830, 1880, 1881, 1882, 1883, 1884, 1885, + 1886, 1887, 1837, 1889, 1890, 1891, 1892, 1893, 1894, 2035, + 1895, 1851, 1897, 1857, 0, 1898, 1018, 112, 1259, 1141, + 593, 1139, 1272, 0, 0, 1324, 0, 0, 0, 0, + 481, 0, 0, 0, 0, 1284, 0, 0, 1606, 201, + 1606, 366, 0, 1606, 366, 1606, 0, 1606, 206, 210, + 1298, 1301, 0, 431, 426, 424, 423, 425, 1606, 253, + 0, 0, 1267, 436, 437, 0, 405, 0, 0, 407, + 0, 0, 217, 0, 215, 0, 401, 155, 0, 228, + 1382, 1383, 1381, 0, 0, 1340, 231, 1376, 1385, 1375, + 1384, 1339, 248, 1334, 0, 0, 1330, 471, 0, 0, + 0, 0, 1148, 877, 876, 860, 861, 874, 875, 862, + 863, 870, 871, 879, 878, 868, 869, 864, 865, 858, + 859, 866, 867, 872, 873, 856, 857, 1161, 1149, 1150, + 1151, 1152, 1153, 1154, 1155, 1156, 1157, 1158, 1159, 1160, + 0, 0, 705, 703, 0, 0, 0, 0, 0, 0, + 1188, 0, 970, 996, 0, 0, 0, 1133, 1171, 0, + 0, 0, 0, 0, 0, 1133, 1177, 0, 0, 728, + 740, 0, 627, 633, 704, 702, 0, 1211, 697, 0, + 774, 754, 0, 753, 0, 0, 756, 750, 0, 751, + 0, 0, 0, 0, 752, 0, 0, 0, 0, 700, + 0, 740, 0, 701, 771, 1416, 1415, 1411, 1398, 1406, + 195, 0, 1263, 1899, 1900, 1901, 818, 1902, 847, 825, + 847, 847, 1903, 1904, 1905, 1906, 814, 814, 827, 1907, + 1908, 1909, 1910, 1911, 815, 816, 852, 1912, 1913, 1914, + 1915, 1916, 0, 0, 1917, 847, 1918, 814, 1919, 1920, + 1921, 819, 1922, 782, 1923, 0, 1924, 817, 783, 1925, + 855, 855, 1926, 0, 842, 1927, 1264, 1265, 0, 1144, + 792, 800, 801, 802, 803, 828, 829, 804, 834, 835, + 805, 899, 0, 814, 155, 1461, 1477, 0, 1138, 1004, + 846, 833, 1186, 0, 841, 840, 0, 1138, 823, 822, + 821, 1009, 0, 820, 1095, 847, 847, 845, 925, 824, + 0, 0, 0, 0, 0, 851, 0, 849, 926, 904, + 905, 0, 1221, 1230, 1133, 1137, 0, 1007, 1133, 0, + 1000, 1001, 0, 1085, 1087, 0, 0, 1189, 1244, 1008, + 0, 1249, 0, 0, 899, 899, 1217, 1115, 0, 1105, + 1108, 0, 0, 1112, 1113, 1114, 0, 0, 0, 1209, + 0, 1123, 1125, 0, 0, 941, 1121, 0, 944, 0, + 0, 0, 0, 1109, 1110, 1111, 1101, 1102, 1103, 1104, + 1106, 1107, 1119, 1100, 922, 0, 997, 0, 1048, 0, + 921, 1215, 695, 0, 1247, 695, 1400, 1404, 1405, 1403, + 1399, 0, 1394, 1393, 1396, 1397, 1479, 0, 1436, 1420, + 0, 1417, 1136, 690, 563, 1235, 0, 567, 1443, 1441, + 160, 159, 0, 0, 536, 535, 602, 594, 596, 602, + 0, 534, 0, 650, 651, 0, 0, 0, 0, 683, + 681, 1243, 1256, 638, 612, 637, 0, 0, 616, 0, + 642, 900, 676, 518, 606, 607, 610, 517, 0, 679, + 0, 689, 0, 555, 557, 540, 554, 552, 537, 545, + 677, 611, 0, 1467, 0, 0, 0, 0, 0, 1606, + 0, 0, 785, 83, 64, 318, 138, 0, 0, 0, + 0, 0, 0, 0, 91, 88, 89, 90, 0, 0, + 0, 0, 1263, 232, 233, 246, 0, 237, 238, 235, + 239, 240, 0, 0, 225, 226, 0, 0, 0, 0, + 224, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1451, 1444, 1190, 1195, 593, 593, 593, 0, 591, 592, + 0, 0, 0, 0, 0, 470, 364, 374, 0, 0, + 0, 1288, 0, 0, 0, 0, 0, 205, 0, 401, + 1289, 1287, 1291, 1290, 1292, 1577, 189, 0, 0, 200, + 197, 0, 363, 337, 0, 0, 1303, 0, 0, 0, + 0, 0, 0, 1606, 353, 1300, 0, 1448, 0, 0, + 251, 438, 1268, 0, 435, 438, 1234, 0, 438, 219, + 0, 0, 1336, 1378, 229, 249, 230, 250, 483, 478, + 508, 0, 486, 491, 468, 0, 468, 0, 488, 492, + 468, 487, 0, 468, 482, 1412, 0, 1041, 0, 1031, + 0, 0, 763, 0, 0, 1032, 972, 973, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 990, 989, 1033, 767, + 0, 770, 0, 0, 1169, 1170, 0, 1034, 0, 0, + 1176, 0, 0, 0, 1039, 0, 707, 0, 0, 0, + 622, 626, 629, 0, 632, 569, 525, 1597, 1637, 0, + 580, 580, 580, 578, 568, 0, 654, 0, 0, 729, + 0, 0, 731, 733, 0, 0, 736, 713, 712, 0, + 0, 0, 0, 775, 0, 1239, 0, 0, 196, 0, + 0, 0, 800, 0, 0, 0, 790, 786, 0, 880, + 881, 882, 883, 884, 885, 886, 887, 888, 889, 890, + 891, 806, 1276, 0, 812, 1281, 1282, 1279, 1275, 1278, + 1280, 1283, 0, 0, 0, 0, 1185, 1181, 0, 0, + 0, 0, 1090, 1092, 1094, 0, 844, 843, 1099, 1105, + 1108, 1112, 1113, 1114, 1109, 1110, 1111, 1101, 1102, 1103, + 1104, 1106, 1107, 0, 1127, 0, 1081, 0, 0, 0, + 0, 0, 0, 1220, 0, 968, 0, 1002, 1013, 0, + 0, 1088, 1017, 1222, 1197, 0, 0, 0, 1252, 1251, + 901, 910, 913, 945, 946, 917, 918, 919, 923, 1274, + 1273, 1216, 0, 1208, 0, 0, 902, 927, 932, 0, + 1178, 962, 0, 950, 0, 940, 0, 948, 952, 928, + 943, 0, 924, 0, 1209, 1124, 1126, 0, 1122, 0, + 914, 915, 916, 906, 907, 908, 909, 911, 912, 920, + 1098, 1096, 1097, 0, 1195, 0, 1207, 0, 0, 1050, + 0, 0, 947, 1213, 0, 774, 593, 774, 0, 899, + 1437, 1271, 1430, 1419, 1271, 1134, 1236, 1270, 565, 0, + 0, 0, 1439, 146, 150, 0, 1196, 180, 182, 695, + 0, 600, 601, 605, 0, 0, 605, 584, 533, 1848, + 1730, 0, 0, 0, 0, 643, 684, 0, 675, 640, + 641, 0, 639, 1243, 644, 1242, 645, 648, 649, 617, + 1231, 685, 687, 0, 680, 0, 1237, 539, 558, 0, + 0, 0, 0, 0, 522, 521, 691, 0, 49, 0, + 1606, 66, 0, 0, 0, 0, 0, 0, 268, 0, + 368, 268, 104, 1606, 438, 1606, 438, 1501, 1572, 1748, + 0, 62, 342, 95, 0, 132, 371, 0, 327, 85, + 100, 125, 0, 0, 222, 51, 236, 241, 128, 245, + 242, 1308, 243, 139, 0, 47, 0, 126, 0, 1306, + 0, 0, 53, 130, 1310, 1452, 0, 1194, 0, 591, + 591, 591, 0, 1140, 0, 0, 0, 1142, 1143, 940, + 1317, 1318, 1319, 1316, 456, 469, 0, 365, 0, 480, + 459, 460, 470, 1286, 0, 201, 366, 0, 366, 0, + 210, 1288, 0, 0, 191, 187, 0, 0, 0, 0, + 364, 356, 354, 387, 0, 361, 355, 0, 0, 313, + 0, 1495, 0, 0, 0, 0, 450, 0, 0, 205, + 211, 0, 0, 0, 253, 254, 404, 1269, 406, 0, + 408, 218, 216, 1331, 475, 0, 474, 479, 473, 477, + 472, 0, 467, 0, 501, 0, 0, 0, 0, 0, + 0, 0, 0, 1028, 1146, 0, 1164, 1163, 971, 978, + 981, 985, 986, 987, 1165, 0, 0, 0, 982, 983, + 984, 974, 975, 976, 977, 979, 980, 988, 772, 0, + 0, 766, 1174, 1173, 1167, 1168, 0, 1036, 1037, 1038, + 1175, 0, 0, 741, 620, 618, 621, 623, 619, 0, + 0, 774, 580, 580, 580, 580, 577, 0, 0, 0, + 773, 0, 671, 735, 0, 759, 0, 732, 0, 0, + 723, 0, 730, 779, 746, 0, 0, 748, 1407, 796, + 0, 791, 787, 0, 0, 0, 797, 0, 0, 0, + 0, 0, 0, 0, 1145, 586, 1005, 0, 0, 0, + 1182, 0, 967, 813, 826, 0, 1093, 1015, 0, 1116, + 1080, 854, 853, 855, 855, 0, 0, 0, 1229, 0, + 1134, 1084, 1086, 1230, 1010, 838, 899, 0, 0, 0, + 0, 0, 0, 0, 951, 942, 0, 949, 953, 0, + 0, 0, 936, 0, 0, 934, 963, 930, 0, 0, + 964, 1194, 0, 1198, 0, 0, 1049, 1058, 698, 694, + 654, 591, 654, 0, 1401, 1421, 1418, 566, 155, 1440, + 0, 169, 0, 0, 0, 0, 172, 186, 183, 1439, + 0, 0, 595, 597, 0, 1117, 605, 599, 647, 646, + 0, 615, 682, 613, 0, 688, 0, 556, 0, 542, + 0, 716, 0, 0, 0, 0, 0, 317, 0, 0, + 0, 268, 0, 376, 0, 383, 0, 0, 368, 349, + 84, 0, 0, 0, 58, 103, 76, 68, 54, 82, + 0, 0, 87, 0, 80, 97, 98, 96, 101, 0, + 278, 303, 0, 0, 314, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 483, 1195, 1191, 1195, + 0, 0, 0, 593, 589, 590, 1019, 0, 455, 507, + 504, 505, 503, 227, 375, 0, 0, 199, 363, 0, + 1303, 0, 0, 1285, 401, 0, 192, 0, 190, 0, + 201, 366, 0, 341, 337, 362, 335, 334, 336, 0, + 1496, 220, 0, 1490, 366, 1302, 0, 0, 451, 0, + 445, 210, 0, 0, 1297, 252, 438, 0, 463, 502, + 509, 489, 494, 0, 500, 496, 495, 490, 498, 497, + 493, 1029, 1040, 1162, 0, 0, 0, 0, 765, 768, + 0, 1035, 1030, 739, 0, 0, 654, 0, 0, 0, + 0, 571, 570, 576, 0, 0, 1052, 734, 0, 0, + 0, 721, 711, 717, 0, 0, 0, 0, 777, 776, + 747, 800, 0, 780, 800, 0, 800, 0, 798, 0, + 807, 892, 893, 894, 895, 896, 897, 898, 832, 0, + 1184, 1180, 1089, 1091, 1128, 850, 848, 1219, 1133, 1224, + 1226, 0, 0, 0, 1083, 969, 1250, 903, 0, 0, + 933, 1179, 954, 0, 0, 0, 929, 1116, 0, 0, + 0, 0, 0, 938, 0, 1202, 1195, 0, 1201, 0, + 0, 0, 0, 1024, 699, 671, 0, 671, 0, 0, + 1438, 0, 1433, 147, 148, 149, 0, 0, 0, 164, + 141, 0, 0, 181, 169, 157, 603, 604, 0, 598, + 614, 1232, 1238, 541, 0, 1007, 0, 0, 538, 0, + 133, 268, 0, 0, 65, 0, 385, 329, 377, 360, + 344, 0, 0, 0, 269, 0, 402, 0, 0, 350, + 0, 0, 0, 0, 330, 0, 0, 289, 0, 0, + 360, 0, 367, 285, 286, 0, 57, 77, 0, 73, + 0, 102, 0, 0, 0, 0, 0, 60, 72, 0, + 55, 0, 438, 438, 63, 1263, 1899, 1900, 1901, 1902, + 1903, 1904, 1905, 1906, 1907, 1908, 2019, 1909, 1910, 1911, + 1912, 1913, 1914, 1915, 1916, 2028, 1917, 275, 1918, 1675, + 1919, 1920, 1921, 1922, 1923, 0, 1924, 783, 1925, 1926, + 2107, 1927, 1101, 1102, 273, 274, 370, 270, 1264, 271, + 378, 272, 0, 373, 328, 129, 1309, 0, 127, 0, + 1307, 136, 134, 131, 1311, 1445, 0, 0, 1022, 1023, + 1020, 591, 0, 0, 0, 483, 462, 0, 0, 1495, + 0, 0, 0, 0, 1606, 0, 188, 0, 1303, 198, + 363, 0, 393, 313, 388, 0, 1495, 1493, 0, 1303, + 1489, 0, 442, 0, 202, 0, 0, 0, 409, 476, + 0, 499, 991, 0, 0, 0, 0, 630, 0, 636, + 671, 575, 574, 573, 572, 653, 1546, 1831, 1729, 0, + 657, 652, 655, 660, 662, 661, 663, 659, 670, 0, + 673, 758, 1129, 1131, 0, 0, 0, 0, 722, 0, + 724, 0, 726, 0, 778, 794, 0, 795, 0, 793, + 788, 799, 1183, 1227, 1228, 1223, 0, 900, 960, 958, + 955, 0, 956, 937, 0, 0, 935, 931, 0, 965, + 0, 0, 1199, 0, 1044, 0, 1047, 1061, 1057, 1056, + 1052, 1019, 1052, 1402, 564, 168, 145, 171, 170, 0, + 1196, 178, 0, 0, 169, 0, 173, 452, 0, 0, + 553, 715, 546, 547, 0, 381, 67, 0, 360, 0, + 268, 346, 345, 348, 343, 347, 0, 403, 0, 0, + 287, 0, 294, 332, 333, 331, 288, 360, 366, 290, + 0, 0, 0, 69, 59, 56, 61, 70, 0, 0, + 71, 74, 779, 86, 79, 1263, 2028, 2037, 0, 0, + 0, 0, 0, 1193, 1192, 0, 458, 457, 506, 454, + 465, 227, 0, 0, 337, 1492, 0, 0, 447, 0, + 0, 0, 363, 193, 0, 0, 0, 1495, 0, 0, + 265, 0, 310, 0, 213, 1494, 0, 0, 1481, 0, + 0, 0, 1295, 1296, 0, 464, 992, 0, 993, 769, + 0, 0, 628, 1052, 0, 0, 0, 664, 658, 0, + 1051, 1053, 0, 625, 1132, 718, 0, 720, 0, 744, + 593, 0, 744, 727, 789, 781, 1225, 1042, 0, 957, + 961, 959, 939, 1195, 1203, 1195, 1200, 1046, 1060, 1063, + 673, 1248, 673, 0, 0, 156, 0, 0, 153, 140, + 158, 1118, 543, 544, 0, 268, 0, 359, 382, 299, + 277, 0, 0, 0, 284, 291, 392, 293, 0, 78, + 94, 0, 0, 372, 137, 135, 1021, 483, 0, 1303, + 313, 1489, 444, 204, 0, 0, 0, 337, 220, 1491, + 326, 319, 320, 321, 322, 323, 324, 325, 340, 339, + 311, 312, 0, 0, 0, 0, 0, 446, 0, 1297, + 0, 175, 184, 0, 175, 994, 631, 0, 673, 0, + 0, 0, 656, 0, 0, 672, 0, 529, 1130, 0, + 710, 708, 591, 0, 709, 0, 0, 0, 0, 593, + 625, 625, 142, 0, 143, 179, 0, 0, 0, 366, + 384, 358, 0, 351, 297, 296, 298, 302, 0, 300, + 0, 316, 0, 309, 277, 0, 81, 0, 379, 453, + 461, 0, 267, 1483, 363, 0, 1489, 313, 1495, 1489, + 0, 1486, 0, 443, 203, 0, 0, 0, 177, 1303, + 0, 177, 0, 625, 666, 0, 665, 1055, 1054, 627, + 719, 0, 0, 1043, 1205, 1204, 0, 1067, 528, 527, + 0, 0, 0, 0, 392, 0, 338, 0, 0, 299, + 0, 292, 389, 390, 391, 0, 305, 295, 306, 75, + 93, 380, 0, 363, 1484, 266, 214, 1482, 1487, 1488, + 0, 175, 174, 602, 176, 774, 185, 602, 635, 530, + 667, 624, 0, 725, 1062, 0, 0, 0, 0, 0, + 152, 774, 163, 0, 309, 357, 352, 276, 301, 315, + 0, 0, 0, 307, 0, 308, 1489, 0, 177, 605, + 1293, 605, 1817, 1547, 1783, 0, 1079, 1068, 1079, 1079, + 1059, 144, 151, 0, 268, 281, 0, 280, 0, 369, + 304, 1485, 1303, 602, 165, 166, 0, 1072, 1071, 1070, + 1074, 1073, 0, 1066, 1064, 1065, 774, 386, 279, 283, + 282, 774, 605, 0, 0, 1076, 0, 1077, 162, 1294, + 167, 1069, 1075, 1078 }; -/* YYR2[YYN] -- Number of symbols composing right hand side of rule YYN. */ -static const yytype_uint8 yyr2[] = +/* YYPGOTO[NTERM-NUM]. */ +static const yytype_int16 yypgoto[] = { - 0, 2, 1, 3, 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, 0, 4, 6, 4, 6, 4, 6, 4, - 6, 1, 2, 3, 2, 1, 3, 2, 3, 1, - 3, 2, 5, 3, 6, 4, 6, 6, 6, 5, - 5, 6, 9, 4, 5, 7, 6, 4, 8, 4, - 2, 4, 3, 6, 4, 2, 2, 2, 2, 1, - 2, 0, 1, 2, 2, 2, 1, 3, 4, 2, - 1, 0, 2, 3, 2, 3, 1, 2, 1, 1, - 1, 1, 1, 1, 1, 2, 2, 1, 1, 1, - 1, 1, 6, 6, 8, 6, 8, 6, 8, 6, - 8, 8, 10, 8, 10, 1, 0, 9, 1, 4, - 4, 7, 2, 1, 3, 2, 2, 0, 4, 3, - 0, 1, 0, 2, 3, 5, 2, 2, 0, 8, - 5, 0, 5, 5, 7, 2, 0, 1, 1, 1, - 3, 2, 0, 1, 0, 1, 3, 1, 3, 1, - 2, 1, 3, 2, 6, 8, 5, 1, 0, 1, - 3, 2, 4, 5, 5, 8, 7, 1, 0, 3, - 9, 12, 1, 3, 1, 3, 3, 0, 4, 6, - 1, 2, 1, 1, 0, 1, 2, 2, 1, 2, - 2, 1, 2, 3, 2, 2, 2, 2, 3, 3, - 3, 1, 3, 1, 0, 1, 2, 2, 2, 2, - 2, 2, 2, 2, 1, 1, 0, 2, 9, 12, - 11, 0, 2, 1, 1, 1, 1, 1, 1, 3, - 0, 1, 2, 1, 1, 2, 2, 3, 1, 1, - 2, 2, 1, 2, 3, 5, 3, 2, 5, 1, - 1, 1, 0, 5, 7, 5, 2, 3, 1, 1, - 2, 2, 0, 3, 4, 4, 0, 3, 2, 0, - 3, 1, 1, 1, 1, 1, 1, 1, 1, 1, - 1, 3, 3, 1, 2, 2, 2, 2, 2, 2, - 0, 3, 3, 3, 0, 1, 2, 1, 2, 2, - 2, 2, 3, 4, 1, 3, 1, 1, 1, 1, - 3, 1, 2, 0, 1, 2, 0, 1, 3, 0, - 2, 0, 3, 3, 1, 5, 3, 1, 3, 1, - 2, 1, 4, 5, 5, 6, 3, 7, 4, 11, - 1, 3, 2, 2, 2, 0, 3, 1, 1, 2, - 2, 2, 2, 1, 0, 1, 2, 6, 4, 6, - 4, 6, 8, 1, 1, 1, 1, 2, 1, 2, - 1, 2, 1, 1, 1, 1, 3, 3, 3, 3, - 1, 2, 2, 1, 3, 1, 1, 1, 3, 1, - 1, 0, 1, 1, 1, 8, 11, 10, 7, 10, - 9, 1, 1, 2, 3, 8, 11, 9, 7, 0, - 3, 3, 1, 1, 3, 0, 1, 3, 1, 0, - 1, 0, 1, 0, 1, 3, 1, 1, 1, 3, - 1, 0, 2, 2, 0, 2, 0, 1, 0, 1, - 1, 1, 3, 3, 1, 1, 3, 3, 3, 3, - 3, 3, 4, 3, 2, 1, 1, 1, 1, 1, - 3, 1, 1, 3, 3, 1, 2, 4, 4, 2, - 3, 5, 5, 1, 1, 3, 0, 11, 11, 10, - 12, 1, 2, 5, 4, 4, 4, 4, 7, 5, - 4, 7, 6, 9, 9, 4, 1, 1, 1, 1, - 1, 1, 1, 5, 1, 1, 3, 1, 2, 2, - 2, 3, 1, 3, 7, 1, 2, 0, 2, 0, - 3, 3, 4, 4, 4, 4, 3, 2, 1, 1, - 0, 1, 1, 0, 2, 1, 5, 1, 0, 2, - 2, 0, 1, 0, 3, 5, 1, 3, 4, 3, - 1, 1, 0, 2, 2, 0, 2, 2, 1, 1, - 1, 0, 2, 4, 5, 4, 2, 3, 2, 2, - 2, 2, 1, 2, 3, 0, 1, 0, 5, 1, - 4, 6, 2, 1, 0, 4, 0, 1, 1, 2, - 2, 2, 1, 1, 2, 2, 1, 1, 1, 1, - 1, 1, 3, 3, 0, 1, 3, 1, 2, 1, - 1, 1, 1, 1, 2, 4, 4, 5, 1, 1, - 2, 0, 2, 0, 1, 3, 1, 0, 1, 2, - 3, 2, 4, 2, 3, 2, 0, 1, 2, 0, - 4, 5, 1, 2, 2, 0, 1, 3, 1, 2, - 3, 3, 3, 3, 3, 3, 1, 4, 9, 9, - 3, 0, 2, 2, 0, 5, 3, 1, 3, 5, - 3, 1, 2, 1, 3, 5, 1, 2, 3, 4, - 5, 4, 5, 4, 6, 5, 4, 5, 5, 5, - 2, 4, 1, 1, 0, 1, 4, 5, 4, 0, - 2, 2, 2, 1, 1, 1, 1, 0, 4, 2, - 1, 2, 2, 4, 2, 6, 2, 1, 3, 4, - 0, 2, 0, 2, 0, 1, 3, 3, 2, 0, - 2, 4, 1, 1, 1, 0, 2, 3, 5, 6, - 2, 3, 1, 5, 5, 5, 3, 3, 3, 4, - 0, 1, 1, 1, 1, 1, 2, 4, 1, 1, - 1, 1, 2, 3, 0, 1, 1, 1, 1, 1, - 2, 2, 2, 2, 2, 1, 3, 0, 1, 1, - 1, 1, 5, 2, 1, 1, 1, 1, 4, 1, - 2, 2, 1, 3, 3, 2, 1, 0, 5, 2, - 5, 2, 1, 3, 3, 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, 3, 3, 3, 3, 3, 3, 0, - 1, 3, 3, 5, 2, 2, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, - 3, 2, 2, 3, 3, 2, 2, 3, 3, 5, - 4, 6, 3, 5, 4, 6, 4, 6, 5, 7, - 3, 2, 4, 3, 2, 3, 3, 3, 3, 4, - 3, 4, 3, 4, 5, 6, 6, 7, 6, 7, - 6, 7, 3, 4, 4, 6, 1, 4, 3, 5, - 1, 3, 2, 2, 3, 3, 3, 3, 3, 3, - 3, 3, 3, 3, 3, 3, 3, 3, 3, 2, - 2, 5, 6, 6, 7, 1, 1, 2, 1, 1, - 2, 2, 3, 1, 2, 4, 1, 1, 2, 2, - 4, 1, 1, 3, 1, 4, 1, 3, 3, 6, - 7, 9, 7, 7, 5, 1, 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, 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, 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, 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, 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, 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, - 8, 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, 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, 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, - 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, 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, 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, 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, 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, - 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, 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 - STATE-NUM when YYTABLE doesn't specify something else to do. Zero - means the default is an error. */ -static const yytype_uint16 yydefact[] = -{ - 152, 246, 0, 1345, 1344, 1414, 246, 0, 1294, 246, - 468, 384, 0, 1435, 1414, 0, 246, 0, 152, 0, - 0, 0, 0, 0, 0, 528, 531, 529, 0, 0, - 0, 246, 568, 0, 1434, 246, 0, 0, 560, 530, - 0, 1393, 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, 504, 491, 573, - 503, 0, 0, 151, 672, 511, 31, 15, 27, 5, - 11, 25, 26, 24, 1303, 39, 29, 37, 20, 8, - 9, 21, 38, 40, 1433, 10, 41, 13, 245, 244, - 238, 0, 0, 0, 0, 0, 1413, 0, 0, 239, - 108, 1459, 1460, 1461, 1462, 1463, 1464, 1465, 1466, 1467, - 1468, 1469, 1833, 1470, 1471, 1472, 1473, 1474, 1834, 1475, - 1476, 1477, 1779, 1780, 1835, 1781, 1782, 1478, 1479, 1480, - 1481, 1482, 1483, 1484, 1485, 1486, 1487, 1783, 1784, 1488, - 1489, 1490, 1491, 1492, 1785, 1836, 1786, 1493, 1494, 1495, - 1496, 1497, 1837, 1498, 1499, 1500, 1501, 1502, 1503, 1504, - 1505, 1506, 1838, 1507, 1508, 1509, 1510, 1511, 1512, 1513, - 1514, 1515, 1516, 1787, 1517, 1518, 1788, 1519, 1520, 1521, - 1522, 1523, 1524, 1525, 1526, 1527, 1528, 1529, 1530, 1531, - 1532, 1533, 1534, 1535, 1536, 1537, 1538, 1539, 1540, 1541, - 1542, 1543, 1544, 1545, 1546, 1789, 1547, 1548, 1549, 1550, - 1551, 1790, 1552, 1553, 1554, 1791, 1555, 1556, 1557, 1839, - 1840, 1558, 1559, 1792, 1842, 1560, 1561, 1793, 1794, 1562, - 1563, 1564, 1565, 1566, 1567, 1568, 1569, 1570, 1843, 1571, - 1572, 1573, 1574, 1575, 1576, 1577, 1578, 1579, 1580, 1581, - 1582, 1844, 1795, 1583, 1584, 1585, 1586, 1587, 1796, 1797, - 1798, 1588, 1845, 1846, 1589, 1847, 1590, 1591, 1592, 1593, - 1594, 1595, 1596, 1848, 1597, 1849, 1598, 1599, 1600, 1601, - 1602, 1603, 1604, 1605, 1799, 1606, 1607, 1608, 1609, 1610, - 1611, 1612, 1613, 1614, 1615, 1616, 1617, 1618, 1619, 1620, - 1621, 1622, 1623, 1624, 1625, 1800, 1851, 1801, 1626, 1627, - 1628, 1802, 1629, 1630, 1852, 1631, 1803, 1632, 1804, 1633, - 1634, 1635, 1636, 1637, 1638, 1639, 1640, 1641, 1642, 1805, - 1853, 1643, 1854, 1806, 1644, 1645, 1646, 1647, 1648, 1649, - 1650, 1651, 1652, 1653, 1654, 1655, 1807, 1855, 1656, 1657, - 1808, 1658, 1659, 1660, 1661, 1662, 1663, 1664, 1665, 1666, - 1667, 1668, 1669, 1809, 1670, 1671, 1672, 1673, 1674, 1675, - 1676, 1677, 1678, 1679, 1680, 1681, 1682, 1683, 1684, 1685, - 1686, 1687, 1688, 1856, 1689, 1690, 1691, 1810, 1692, 1693, - 1694, 1695, 1696, 1697, 1698, 1699, 1700, 1701, 1702, 1703, - 1704, 1705, 1706, 1707, 1708, 1811, 1709, 1710, 1711, 1857, - 1712, 1713, 1812, 1714, 1715, 1716, 1717, 1718, 1719, 1720, - 1721, 1722, 1723, 1724, 1725, 1726, 1813, 1727, 1814, 1728, - 1729, 1730, 1731, 1859, 1732, 1733, 1734, 1735, 1736, 1737, - 1815, 1816, 1738, 1739, 1817, 1740, 1818, 1741, 1742, 1819, - 1743, 1744, 1745, 1746, 1747, 1748, 1749, 1750, 1751, 1752, - 1753, 1754, 1755, 1756, 1757, 1758, 1759, 1820, 1821, 1760, - 1860, 1761, 1762, 1763, 1764, 1765, 1766, 1767, 1768, 1769, - 1770, 1771, 1772, 1773, 1774, 1822, 1823, 1824, 1825, 1826, - 1827, 1828, 1829, 1830, 1831, 1832, 1775, 1776, 1777, 1778, - 0, 1442, 0, 1220, 109, 110, 1242, 108, 1792, 1799, - 1813, 1293, 1292, 109, 241, 467, 0, 0, 0, 0, - 0, 0, 0, 378, 377, 0, 1283, 383, 0, 0, - 112, 104, 1658, 111, 1219, 102, 1413, 0, 0, 402, - 403, 0, 413, 0, 395, 400, 396, 0, 422, 415, - 423, 404, 394, 416, 405, 393, 0, 424, 410, 398, - 0, 0, 0, 242, 207, 152, 0, 1309, 1319, 1328, - 1324, 1318, 1326, 1316, 1322, 1308, 1330, 1317, 1321, 1314, - 1331, 1312, 1329, 1327, 1315, 1323, 1307, 1311, 1298, 1303, - 1334, 1325, 1332, 1320, 1333, 1335, 1310, 1336, 1313, 0, - 1294, 0, 0, 1785, 1836, 1790, 0, 1803, 0, 1806, - 1807, 1692, 1814, 1817, 1818, 1819, 1820, 0, 740, 111, - 106, 724, 0, 506, 676, 686, 724, 729, 1006, 752, - 1007, 0, 113, 1379, 1378, 1374, 1373, 191, 1257, 1422, - 1560, 1600, 1707, 1815, 1738, 1438, 1423, 1417, 1421, 243, - 567, 565, 0, 1191, 1560, 1600, 1696, 1707, 1815, 1353, - 1357, 0, 240, 1440, 1427, 0, 1428, 111, 512, 559, - 0, 247, 1392, 0, 1397, 0, 1672, 539, 542, 1251, - 540, 504, 0, 1, 152, 0, 158, 0, 563, 563, - 0, 563, 0, 496, 0, 0, 504, 499, 503, 673, - 1302, 1407, 1437, 1815, 1738, 1426, 1429, 1569, 0, 0, - 1569, 0, 1569, 0, 1569, 0, 0, 1416, 1175, 0, - 1221, 114, 0, 0, 0, 349, 382, 381, 380, 379, - 384, 1569, 1267, 0, 431, 432, 0, 0, 0, 0, - 0, 1278, 105, 103, 0, 1411, 411, 412, 0, 401, - 397, 399, 0, 1569, 1246, 421, 417, 1569, 421, 1213, - 1569, 0, 0, 199, 1300, 1337, 1890, 1891, 1892, 1893, - 1894, 1895, 1896, 1897, 1899, 1898, 1900, 1901, 1902, 1903, - 1904, 1905, 1906, 1907, 1908, 1909, 1910, 1911, 1912, 1913, - 1914, 1915, 1916, 1917, 1918, 1921, 1919, 1920, 1922, 1923, - 1924, 1925, 1926, 1927, 1928, 1929, 1930, 1931, 1932, 1933, - 1934, 1936, 1935, 1937, 1938, 1939, 1940, 1941, 1942, 1943, - 1944, 1945, 1946, 1947, 1948, 1949, 1950, 1951, 1952, 1953, - 1954, 1955, 1956, 1957, 1958, 1959, 1960, 1961, 1962, 1963, - 1964, 1965, 1966, 1967, 1968, 1969, 1970, 1351, 0, 1352, - 1342, 1306, 1338, 1339, 152, 0, 466, 1291, 1375, 0, - 0, 0, 1127, 724, 729, 0, 0, 742, 0, 1146, - 0, 1152, 0, 0, 0, 724, 511, 0, 686, 741, - 107, 0, 722, 723, 614, 614, 568, 0, 549, 734, - 0, 0, 737, 735, 0, 737, 0, 0, 0, 737, - 733, 694, 0, 614, 0, 722, 725, 614, 0, 744, - 1297, 0, 0, 0, 0, 0, 1420, 1418, 1419, 1424, - 0, 0, 0, 1223, 1225, 1226, 1095, 1236, 992, 0, - 1780, 1781, 1782, 1167, 1783, 1784, 1786, 1787, 1788, 946, - 1534, 1789, 1234, 1791, 1793, 1794, 1796, 1797, 1798, 1799, - 1800, 1801, 0, 1235, 1804, 1638, 1809, 1810, 1812, 1815, - 1816, 1233, 1821, 0, 0, 0, 1202, 1118, 0, 0, - 0, 991, 0, 1168, 1176, 987, 0, 0, 788, 789, - 810, 811, 790, 816, 817, 819, 791, 0, 1198, 880, - 975, 1186, 994, 1025, 996, 1005, 983, 1062, 976, 0, - 986, 978, 1194, 549, 1192, 0, 979, 1222, 549, 1190, - 1356, 1354, 1360, 1355, 0, 0, 0, 0, 0, 107, - 1400, 1399, 1391, 1389, 1390, 1388, 1387, 1394, 0, 1396, - 1303, 1113, 1115, 0, 541, 0, 0, 0, 494, 493, - 3, 0, 0, 0, 0, 561, 562, 0, 0, 0, - 0, 0, 0, 0, 0, 657, 588, 589, 591, 654, - 658, 666, 0, 0, 0, 0, 0, 500, 0, 1251, - 1436, 1432, 1430, 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, 1414, 0, 466, 1174, 0, 118, 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, 2044, - 2045, 2046, 2047, 2048, 2049, 2050, 2051, 2052, 2053, 2054, - 2055, 2056, 2057, 2058, 2059, 2060, 2061, 2062, 2063, 2064, - 2065, 2066, 2067, 2068, 2069, 1957, 2070, 2071, 2072, 2073, - 2074, 116, 117, 120, 119, 121, 115, 108, 0, 1971, - 1833, 1834, 1835, 1836, 1786, 1837, 1838, 0, 1839, 1840, - 1792, 1842, 1843, 1844, 1845, 1846, 1847, 1848, 1849, 1799, - 1851, 1852, 1853, 1854, 1855, 1856, 1997, 1857, 1813, 1859, - 1819, 0, 1860, 998, 1121, 573, 1119, 1252, 0, 109, - 1239, 0, 0, 0, 464, 0, 0, 0, 1263, 0, - 0, 1569, 198, 1569, 349, 0, 1569, 349, 1569, 0, - 1277, 1280, 0, 414, 409, 407, 406, 408, 0, 0, - 1247, 419, 420, 0, 388, 0, 0, 390, 0, 0, - 204, 0, 202, 0, 152, 0, 215, 1347, 1348, 1346, - 0, 0, 1341, 1305, 218, 235, 1350, 1340, 1349, 1304, - 1299, 0, 0, 1295, 454, 0, 0, 0, 0, 1128, - 857, 856, 840, 841, 854, 855, 842, 843, 850, 851, - 859, 858, 848, 849, 844, 845, 838, 839, 846, 847, - 852, 853, 836, 837, 1141, 1129, 1130, 1131, 1132, 1133, - 1134, 1135, 1136, 1137, 1138, 1139, 1140, 0, 0, 685, - 683, 0, 0, 0, 0, 0, 0, 1168, 0, 950, - 976, 0, 0, 0, 1113, 1151, 0, 0, 0, 0, - 0, 0, 1113, 1157, 0, 0, 708, 720, 0, 607, - 613, 684, 682, 0, 1191, 677, 0, 754, 0, 734, - 0, 733, 0, 0, 736, 730, 0, 731, 0, 0, - 0, 0, 732, 0, 0, 0, 0, 0, 680, 0, - 720, 0, 681, 751, 1381, 1380, 1376, 1363, 1371, 192, - 0, 1243, 1861, 1862, 1863, 798, 1864, 827, 805, 827, - 827, 1865, 1866, 1867, 1868, 794, 794, 807, 1869, 1870, - 1871, 1872, 1873, 795, 796, 832, 1874, 1875, 1876, 1877, - 1878, 0, 0, 1879, 827, 1880, 794, 1881, 1882, 1883, - 799, 1884, 762, 1885, 0, 1886, 797, 763, 1887, 835, - 835, 1888, 0, 822, 1889, 0, 1124, 772, 780, 781, - 782, 783, 808, 809, 784, 814, 815, 785, 879, 0, - 794, 1244, 1245, 152, 1425, 1439, 0, 1118, 984, 826, - 813, 1166, 0, 821, 820, 0, 1118, 803, 802, 801, - 989, 0, 800, 1075, 827, 827, 825, 905, 804, 0, - 0, 0, 0, 0, 831, 0, 829, 906, 884, 885, - 0, 1201, 1210, 1113, 1117, 0, 987, 1113, 0, 980, - 981, 0, 1065, 1067, 0, 0, 1169, 1224, 988, 0, - 1229, 0, 0, 879, 879, 1197, 1095, 0, 1085, 1088, - 0, 0, 1092, 1093, 1094, 0, 0, 0, 1189, 0, - 1103, 1105, 0, 0, 921, 1101, 0, 924, 0, 0, - 0, 0, 1089, 1090, 1091, 1081, 1082, 1083, 1084, 1086, - 1087, 1099, 1080, 902, 0, 977, 0, 1028, 0, 901, - 1195, 675, 0, 1227, 675, 1365, 1369, 1370, 1364, 1368, - 0, 1359, 1358, 1361, 1362, 1441, 0, 1401, 1385, 0, - 1382, 1116, 670, 543, 1215, 0, 547, 1406, 157, 156, - 0, 0, 516, 515, 582, 574, 576, 582, 0, 514, - 0, 630, 631, 0, 0, 0, 0, 663, 661, 1223, - 1236, 618, 592, 617, 0, 0, 596, 0, 622, 880, - 656, 498, 586, 587, 590, 497, 0, 659, 0, 669, - 0, 535, 537, 520, 534, 532, 517, 525, 657, 591, - 0, 1408, 1431, 0, 0, 0, 0, 0, 1569, 0, - 0, 765, 80, 61, 301, 135, 0, 0, 0, 0, - 0, 0, 0, 88, 85, 86, 87, 0, 0, 0, - 0, 1243, 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, 0, 1415, - 1409, 1170, 1175, 573, 573, 573, 0, 571, 572, 0, - 0, 0, 453, 347, 357, 0, 0, 0, 1267, 0, - 0, 0, 0, 0, 0, 384, 1270, 1268, 1266, 1269, - 1271, 1540, 186, 0, 0, 194, 197, 0, 346, 320, - 0, 0, 1282, 0, 0, 0, 1569, 336, 1279, 0, - 1412, 421, 1248, 0, 418, 421, 1214, 0, 421, 206, - 0, 0, 1301, 1343, 216, 236, 217, 237, 466, 461, - 489, 0, 469, 474, 451, 0, 451, 0, 471, 475, - 451, 470, 0, 451, 465, 1377, 0, 1021, 0, 1011, - 0, 0, 743, 0, 0, 1012, 952, 953, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 970, 969, 1013, 747, - 0, 750, 0, 0, 1149, 1150, 0, 1014, 0, 0, - 1156, 0, 0, 0, 1019, 0, 687, 0, 0, 0, - 602, 606, 609, 0, 612, 549, 505, 1560, 1600, 0, - 560, 560, 560, 558, 548, 0, 634, 0, 0, 0, - 709, 0, 0, 711, 713, 0, 0, 716, 0, 693, - 692, 0, 0, 0, 0, 755, 0, 1219, 0, 0, - 193, 0, 0, 0, 780, 0, 0, 0, 770, 766, - 0, 860, 861, 862, 863, 864, 865, 866, 867, 868, - 869, 870, 871, 786, 1256, 0, 792, 1259, 1260, 1261, - 1258, 1255, 1262, 0, 0, 0, 0, 1165, 1161, 0, - 0, 0, 0, 1070, 1072, 1074, 0, 824, 823, 1079, - 1085, 1088, 1092, 1093, 1094, 1089, 1090, 1091, 1081, 1082, - 1083, 1084, 1086, 1087, 0, 1107, 0, 1061, 0, 0, - 0, 0, 0, 0, 1200, 0, 948, 0, 982, 993, - 0, 0, 1068, 997, 1202, 1177, 0, 0, 0, 1232, - 1231, 881, 890, 893, 925, 926, 897, 898, 899, 903, - 1254, 1253, 1196, 0, 1188, 0, 0, 882, 907, 912, - 0, 1158, 942, 0, 930, 0, 920, 0, 928, 932, - 908, 923, 0, 904, 0, 1189, 1104, 1106, 0, 1102, - 0, 894, 895, 896, 886, 887, 888, 889, 891, 892, - 900, 1078, 1076, 1077, 0, 1175, 0, 1187, 0, 0, - 1030, 0, 0, 927, 1193, 0, 754, 573, 754, 0, - 879, 1402, 1251, 1395, 1251, 1384, 1114, 1216, 1250, 545, - 0, 0, 0, 1404, 143, 147, 0, 1176, 177, 179, - 675, 0, 580, 581, 585, 0, 0, 585, 564, 513, - 1810, 1692, 0, 0, 0, 0, 623, 664, 0, 655, - 620, 621, 0, 619, 1223, 624, 1222, 625, 628, 629, - 597, 1211, 665, 667, 0, 660, 0, 1217, 519, 538, - 0, 0, 0, 0, 0, 502, 501, 671, 0, 46, - 0, 1569, 63, 0, 0, 0, 0, 0, 0, 251, - 0, 351, 251, 101, 1569, 421, 1569, 421, 1463, 1535, - 1708, 0, 59, 325, 92, 0, 129, 354, 0, 310, - 82, 97, 122, 0, 0, 48, 209, 223, 228, 125, - 232, 229, 1287, 230, 136, 0, 44, 0, 123, 0, - 1285, 0, 0, 50, 127, 1289, 1416, 0, 1174, 0, - 571, 571, 571, 0, 1120, 0, 0, 0, 1122, 1123, - 439, 452, 0, 348, 0, 463, 442, 443, 453, 1265, - 0, 198, 349, 0, 349, 0, 1267, 0, 0, 188, - 184, 0, 0, 0, 0, 347, 339, 337, 370, 0, - 344, 338, 0, 0, 296, 0, 1457, 0, 0, 0, - 0, 433, 0, 0, 0, 0, 387, 1249, 389, 0, - 391, 205, 203, 1296, 458, 0, 457, 462, 460, 456, - 455, 0, 450, 0, 484, 0, 0, 0, 0, 0, - 0, 0, 0, 1008, 1126, 0, 1144, 1143, 951, 958, - 961, 965, 966, 967, 1145, 0, 0, 0, 962, 963, - 964, 954, 955, 956, 957, 959, 960, 968, 752, 0, - 0, 746, 1154, 1153, 1147, 1148, 0, 1016, 1017, 1018, - 1155, 0, 0, 721, 600, 598, 601, 603, 599, 0, - 0, 754, 560, 560, 560, 560, 557, 0, 0, 0, - 753, 0, 651, 717, 715, 0, 739, 0, 712, 0, - 718, 0, 703, 0, 710, 759, 726, 0, 0, 728, - 1372, 776, 0, 771, 767, 0, 0, 0, 777, 0, - 0, 0, 0, 0, 0, 0, 1125, 566, 985, 0, - 0, 0, 1162, 0, 947, 793, 806, 0, 1073, 995, - 0, 1096, 1060, 834, 833, 835, 835, 0, 0, 0, - 1209, 0, 1114, 1064, 1066, 1210, 990, 818, 879, 0, - 0, 0, 0, 0, 0, 0, 931, 922, 0, 929, - 933, 0, 0, 0, 916, 0, 0, 914, 943, 910, - 0, 0, 944, 1174, 0, 1178, 0, 0, 1029, 1038, - 678, 674, 634, 571, 634, 0, 1366, 1386, 1383, 546, - 152, 1405, 0, 166, 0, 0, 0, 0, 169, 183, - 180, 1404, 0, 0, 575, 577, 0, 1097, 585, 579, - 627, 626, 0, 595, 662, 593, 0, 668, 0, 536, - 0, 522, 0, 696, 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, 466, 1175, - 1171, 1175, 0, 0, 0, 573, 569, 570, 999, 0, - 438, 487, 488, 486, 214, 358, 0, 0, 196, 346, - 0, 1282, 0, 1264, 384, 0, 189, 0, 187, 0, - 198, 349, 0, 324, 320, 345, 318, 317, 319, 0, - 1458, 207, 0, 1452, 349, 1281, 0, 0, 434, 0, - 428, 0, 1276, 421, 0, 446, 485, 490, 472, 477, - 0, 483, 479, 478, 473, 481, 480, 476, 1009, 1020, - 1142, 0, 0, 0, 0, 745, 748, 0, 1015, 1010, - 719, 0, 0, 634, 0, 0, 0, 0, 551, 550, - 556, 0, 0, 1032, 714, 0, 0, 0, 701, 691, - 697, 0, 0, 0, 0, 757, 756, 727, 780, 0, - 760, 780, 0, 780, 0, 778, 0, 787, 872, 873, - 874, 875, 876, 877, 878, 812, 0, 1164, 1160, 1069, - 1071, 1108, 830, 828, 1199, 1113, 1204, 1206, 0, 0, - 0, 1063, 949, 1230, 883, 0, 0, 913, 1159, 934, - 0, 0, 0, 909, 1096, 0, 0, 0, 0, 0, - 918, 0, 1182, 1175, 0, 1181, 0, 0, 0, 0, - 1004, 679, 651, 0, 651, 0, 0, 1403, 0, 1398, - 144, 145, 146, 0, 0, 0, 161, 138, 0, 0, - 178, 166, 154, 583, 584, 0, 578, 594, 1212, 1218, - 521, 0, 987, 0, 0, 518, 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, 421, - 421, 60, 1243, 1861, 1862, 1863, 1864, 1865, 1866, 1867, - 1868, 1869, 1870, 1981, 1871, 1872, 1873, 1874, 1875, 1876, - 1877, 1878, 1990, 1879, 258, 1880, 1638, 1881, 1882, 1883, - 1884, 1885, 0, 1886, 763, 1887, 1888, 2066, 1889, 1081, - 1082, 257, 256, 353, 253, 361, 255, 0, 1244, 254, - 356, 311, 126, 1288, 0, 124, 0, 1286, 133, 131, - 128, 1290, 1410, 0, 0, 1002, 1003, 1000, 571, 0, - 0, 466, 445, 0, 0, 1457, 0, 0, 0, 1569, - 0, 185, 0, 1282, 195, 346, 0, 376, 296, 371, - 0, 1457, 1455, 0, 1282, 1451, 0, 425, 0, 0, - 0, 392, 459, 0, 482, 971, 0, 0, 0, 0, - 610, 0, 616, 651, 555, 554, 553, 552, 633, 1508, - 1793, 1691, 0, 637, 632, 635, 640, 642, 641, 643, - 639, 650, 0, 653, 738, 1109, 1111, 0, 0, 0, - 0, 702, 0, 704, 0, 706, 0, 758, 774, 0, - 775, 0, 773, 768, 779, 1163, 1207, 1208, 1203, 0, - 880, 940, 938, 935, 0, 936, 917, 0, 0, 915, - 911, 0, 945, 0, 0, 1179, 0, 1024, 0, 1027, - 1041, 1037, 1036, 1032, 999, 1032, 1367, 544, 165, 142, - 168, 167, 0, 1176, 175, 0, 0, 166, 0, 170, - 435, 0, 0, 533, 695, 526, 527, 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, 759, 83, 76, 1243, 1990, - 1999, 0, 0, 0, 0, 0, 1173, 1172, 0, 441, - 440, 437, 448, 214, 0, 0, 320, 1454, 0, 0, - 430, 0, 0, 346, 190, 0, 0, 0, 1457, 0, - 0, 248, 0, 293, 0, 200, 1456, 0, 0, 1443, - 0, 0, 1274, 1275, 0, 447, 972, 0, 973, 749, - 0, 0, 608, 1032, 0, 0, 0, 644, 638, 0, - 1031, 1033, 0, 605, 1112, 698, 0, 700, 0, 724, - 573, 0, 724, 707, 769, 761, 1205, 1022, 0, 937, - 941, 939, 919, 1175, 1183, 1175, 1180, 1026, 1040, 1043, - 653, 1228, 653, 0, 0, 153, 0, 0, 150, 137, - 155, 1098, 523, 524, 0, 251, 0, 342, 365, 282, - 260, 0, 0, 0, 267, 274, 375, 276, 0, 75, - 91, 0, 0, 355, 134, 132, 1001, 466, 0, 1282, - 296, 1451, 427, 0, 0, 0, 320, 207, 1453, 309, - 302, 303, 304, 305, 306, 307, 308, 323, 322, 294, - 295, 0, 0, 0, 0, 0, 429, 1276, 0, 172, - 181, 0, 172, 974, 611, 0, 653, 0, 0, 0, - 636, 0, 0, 652, 0, 509, 1110, 0, 690, 688, - 571, 0, 689, 0, 0, 0, 0, 573, 605, 605, - 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, 436, 444, 0, - 250, 1445, 346, 0, 1451, 296, 1457, 1451, 0, 1448, - 0, 426, 0, 0, 0, 174, 1282, 0, 174, 0, - 605, 646, 0, 645, 1035, 1034, 607, 699, 0, 0, - 1023, 1185, 1184, 0, 1047, 508, 507, 0, 0, 0, - 0, 375, 0, 321, 0, 0, 282, 0, 275, 372, - 373, 374, 0, 288, 278, 289, 72, 90, 363, 0, - 346, 1446, 249, 201, 1444, 1449, 1450, 0, 172, 171, - 582, 173, 754, 182, 582, 615, 510, 647, 604, 0, - 705, 1042, 0, 0, 0, 0, 0, 149, 754, 160, - 0, 292, 340, 335, 259, 284, 298, 0, 0, 0, - 290, 0, 291, 1451, 0, 174, 585, 1272, 585, 1779, - 1509, 1745, 0, 1059, 1048, 1059, 1059, 1039, 141, 148, - 0, 251, 264, 0, 263, 0, 352, 287, 1447, 1282, - 582, 162, 163, 0, 1052, 1051, 1050, 1054, 1053, 0, - 1046, 1044, 1045, 754, 369, 262, 266, 265, 754, 585, - 0, 0, 1056, 0, 1057, 159, 1273, 164, 1049, 1055, - 1058 + -3054, -3054, -3054, 1895, 76, 91, -888, -1189, -927, -1197, + -3054, -111, 94, -3054, -3054, 179, -3054, 1019, -3054, 182, + -794, 546, -3054, 1111, -3054, -3054, 1558, 519, 95, 100, + 2012, -3054, 982, -3054, 1463, 2517, 3539, 1174, -513, -878, + -1251, 18, 17, -3054, -3054, -3054, -3054, -3054, -811, 494, + -1194, -3054, 441, -3054, -3054, -3054, -3054, -193, 168, -2233, + 1, -2149, -2891, -556, -3054, -662, -3054, -311, -3054, -607, + -3054, -864, -613, -648, -2808, -1129, -3054, 1613, -360, -3054, + 595, -3054, -2526, -3054, -3054, 584, -3054, -1158, -3054, -2184, + 158, -589, -2493, -2529, -2153, -906, 236, -595, 219, -2093, + -1186, -3054, 611, -3054, -579, -3054, -881, -1998, 102, -2749, + -1650, -7, -3054, -3054, -3054, -703, -3054, -2495, 108, 1534, + -2100, 110, -1615, -3054, -3054, -1672, 41, -2080, 2, -2087, + 120, -3054, -3054, 123, 33, -578, 288, -3054, 125, -3054, + -3054, 1513, -902, -3054, -3054, 500, 129, 976, -3054, -2545, + -3054, -597, -3054, -626, -615, -3054, -3054, 42, -895, 1499, + -3054, -3054, -3054, 8, -3054, -460, -3054, -3054, -2559, -3054, + 38, -3054, -3054, -3054, -3054, -3054, -395, 305, -3054, -3054, + -536, -2556, -488, -3054, -2974, -2614, -3054, -3054, -636, -2922, + -2067, 130, 78, 1514, 132, 133, -3054, 320, 136, -1136, + 36, 2773, -19, -36, -3054, -32, -3054, -3054, -3054, 651, + -3054, -3054, 22, 53, 1714, -3054, -1010, -3054, -1591, 533, + -3054, 1862, 1865, -2204, -868, -77, -3054, 691, -1654, -2088, + -623, 1140, 1695, 1709, 467, -2190, -3054, -511, -3054, 253, + -3054, -3054, 698, 1185, -1547, -1542, -3054, -2084, -3054, -429, + -316, -3054, -3054, -3054, -3054, -3054, -2483, -2348, -604, 1153, + -3054, 1712, -3054, -3054, -3054, -3054, 7, -1509, 2862, 736, + -84, -3054, -3054, -3054, -3054, 171, -3054, 925, -138, -3054, + 2103, -663, -790, 1915, 140, 251, -1667, 30, 2108, 512, + -3054, -3054, 516, -2045, -1405, 470, -226, 923, -3054, -3054, + -1223, -3054, -1876, -1161, -3054, -3054, -677, 1233, -3054, -3054, + -3054, 1739, 2075, -3054, -3054, 2125, 2141, -3054, -872, 2276, + -752, -1015, 1928, -925, 1932, -929, -931, -954, 1933, 1940, + 1942, 1943, 1944, 1945, -1525, 4806, 325, 4112, -2202, -3054, + 1043, 25, -3054, -1387, 37, -3054, -3054, -3054, -3054, -2280, + -3054, -399, -3054, -397, -3054, -3054, -3054, -1655, -3053, -1678, + -3054, 4206, 858, -3054, -3054, 462, -3054, -3054, -3054, -3054, + -1545, -3054, 5585, 765, -3054, -1982, -3054, -3054, -969, -840, + -1109, -991, -1181, -1913, -3054, -3054, -3054, -3054, -3054, -3054, + -1481, -1750, -211, 824, -3054, -3054, 913, -3054, -3054, -3054, + -1708, -2105, -3054, -3054, -3054, 840, 1515, 148, -823, -1597, + -3054, 882, -2345, -3054, -3054, 463, -3054, -593, -1106, -2410, + 327, -3, -3054, -861, -2489, -3054, -3054, -728, -2617, -1086, + -896, -3054, 137, 1229, 39, -3054, 141, 1278, -3054, -3054, + -3054, 143, -3054, 403, 144, 444, -3054, 973, -3054, 667, + 23, -3054, -3054, -3054, 49, 2849, 11, -2351 }; /* YYDEFGOTO[NTERM-NUM]. */ static const yytype_int16 yydefgoto[] = { - -1, 46, 47, 48, 577, 2528, 2529, 2530, 2181, 1093, - 3284, 2182, 1094, 1095, 2532, 578, 628, 983, 630, 984, - 1558, 731, 1201, 1202, 579, 1707, 580, 2746, 2105, 2476, - 3266, 53, 3014, 2108, 1053, 3017, 3230, 2739, 3012, 2477, - 3305, 3360, 3015, 2109, 2110, 3231, 2111, 581, 2587, 2588, - 582, 583, 1785, 584, 1290, 1291, 773, 585, 1786, 1735, - 2862, 1113, 1725, 1304, 586, 100, 60, 587, 2517, 2863, - 3277, 2543, 3416, 2799, 2800, 3274, 3275, 2520, 2184, 3343, - 3344, 2600, 1716, 3338, 2256, 3217, 2188, 2169, 2801, 2264, - 3177, 2907, 2185, 2781, 2257, 3270, 1798, 2258, 3271, 3033, - 2259, 1764, 1789, 2521, 3345, 2189, 1765, 2516, 2864, 1704, - 2260, 3281, 2261, 538, 2785, 588, 570, 571, 765, 1284, - 572, 589, 750, 1795, 590, 591, 2570, 2238, 3083, 2614, - 3084, 2293, 2232, 1313, 2287, 1820, 1767, 1314, 528, 1834, - 2615, 2574, 1821, 592, 985, 68, 69, 898, 70, 3027, - 71, 72, 1681, 1682, 1683, 675, 687, 688, 2101, 1397, - 1904, 680, 1057, 1650, 662, 663, 2227, 702, 1758, 1645, - 1646, 2114, 2484, 1674, 1675, 1066, 1067, 1892, 3245, 1893, - 1894, 1390, 1391, 3122, 1662, 1666, 1667, 2135, 2125, 1653, - 2362, 2943, 2944, 2945, 2946, 2947, 2948, 2949, 986, 2653, - 3133, 1670, 1671, 1069, 1070, 1071, 1679, 2145, 74, 75, - 2086, 2460, 2461, 634, 2960, 1416, 1684, 2657, 2658, 2659, - 2964, 2965, 2966, 635, 893, 894, 917, 912, 1405, 1913, - 636, 637, 1869, 1870, 2331, 919, 1906, 1924, 1925, 2665, - 2386, 1485, 2170, 1486, 1487, 1939, 1488, 987, 1489, 1517, - 988, 1522, 1491, 989, 990, 991, 1494, 992, 993, 994, - 995, 1510, 996, 997, 1534, 1941, 1942, 1943, 1944, 1945, - 1946, 1947, 1948, 1949, 1950, 1951, 1952, 1953, 1041, 1685, - 999, 1000, 1001, 1002, 1003, 1004, 639, 1005, 1006, 1607, - 2080, 2459, 2953, 3130, 3131, 2730, 3002, 3159, 3257, 3375, - 3403, 3404, 3430, 1007, 1008, 1552, 1553, 1554, 1973, 1974, - 1975, 1976, 2074, 1601, 1602, 1009, 2866, 1604, 1996, 2956, - 2957, 1042, 1383, 1545, 1245, 1246, 1499, 1357, 1358, 1364, - 1844, 1372, 1376, 1874, 1875, 1384, 2042, 1010, 1967, 1968, - 2403, 1512, 1011, 1126, 1557, 2725, 2077, 1605, 2036, 1018, - 1012, 1019, 1014, 1541, 1542, 2420, 2697, 2698, 2006, 2142, - 1634, 2147, 2148, 769, 1015, 1016, 1017, 1247, 513, 1500, - 3361, 1280, 1047, 1248, 2032, 593, 925, 1961, 594, 1258, - 1778, 595, 3113, 2920, 1271, 1799, 2269, 539, 596, 597, - 522, 81, 82, 83, 711, 1306, 598, 1307, 1308, 858, - 84, 2616, 860, 861, 600, 669, 670, 1428, 1621, 1429, - 601, 645, 1426, 602, 1037, 684, 1038, 1040, 603, 1031, - 2473, 2103, 89, 90, 91, 108, 1124, 604, 656, 657, - 605, 94, 1081, 658, 676, 606, 607, 3109, 608, 2603, - 1265, 523, 515, 516, 1502, 1204, 1250, 1205 -}; - -/* YYPACT[STATE-NUM] -- Index in YYTABLE of the portion describing - STATE-NUM. */ -#define YYPACT_NINF -3027 -static const int yypact[] = -{ - 6367, -28, 55, -3027, -3027, 326, -28, 47417, 62162, -28, - 110, 1570, 49385, -3027, 263, 7916, -28, 52337, 68469, 288, - 287, 30902, 332, 52829, 52829, -3027, -3027, -3027, 62162, 52337, - 53321, -28, 333, 62654, -3027, -28, 32870, 49877, 4, -3027, - 52337, 28, 192, 53813, 52337, 3115, 694, 245, -3027, -3027, - -3027, -3027, -3027, 158, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, 115, -3027, 117, - 116, 30902, 30902, 1382, 259, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 323, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 32378, -3027, -3027, -3027, -3027, -3027, - -3027, 54305, 52337, 54797, 50369, 55289, -3027, 519, 810, -3027, - 169, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, 215, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 216, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, 220, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - 386, -3027, 328, -3027, 227, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, 744, 3115, 52337, 801, - 881, 528, 55781, -3027, -3027, 52337, -3027, -3027, 74, 735, - -3027, -3027, 50861, -3027, -3027, -3027, 732, 949, 752, -3027, - -3027, 581, -3027, 153, -3027, -3027, 616, 574, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, 669, -3027, -3027, -3027, - 63146, 56273, 56765, -3027, 551, 6647, 29916, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, 323, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, 52829, - 62162, 52829, 564, 570, 929, 613, 31394, 635, 33363, 645, - 650, 997, 663, 692, 715, 734, 192, 30409, 708, 386, - -3027, 57257, 57257, -53, 4043, -3027, 57257, 57749, -3027, 774, - -3027, 810, -3027, -3027, -3027, 1094, -3027, -92, 776, -3027, - 58241, 58241, 58241, 785, 1071, -3027, -3027, -3027, 807, -3027, - -3027, 1018, 19715, 19715, 63638, 63638, 810, 63638, 819, -3027, - -3027, 42, -3027, -3027, -3027, 1382, 830, 386, -3027, -3027, - 49877, -3027, -3027, 309, 1185, 19715, 52337, 834, -3027, 859, - 834, 883, 941, -3027, 6367, 1307, 1186, 49877, 339, 339, - 1434, 339, 965, 1006, 1846, 3587, -3027, 1762, -3027, 990, - -3027, 52337, 1091, 1013, 1289, -3027, 830, 1366, 824, 1172, - 1386, 7979, 1390, 832, 1392, 955, 1407, 1530, 19715, 44957, - 386, -3027, 10945, 62162, 1016, 1064, -3027, -3027, -3027, -3027, - 1292, 1321, -3027, 1561, -3027, -3027, 1144, 58733, 59225, 59717, - 60209, 1524, -3027, -3027, 1466, -3027, -3027, -3027, 1145, -3027, - -3027, -3027, 257, 1473, 1128, 89, -3027, 1487, 101, -3027, - 1491, 1367, 14047, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, 192, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, 701, -3027, - -3027, 42028, -3027, -3027, 6647, 1234, 1153, -3027, 1501, 19715, - 19715, 1165, 5612, 57257, 57749, 19715, 52337, -3027, 19715, 24368, - 1169, 19715, 19715, 11462, 19715, 28932, 57257, 4043, 1192, -3027, - 671, 52337, 1197, -3027, 1276, 1276, 333, 30902, 1493, -3027, - 913, 1492, 1428, -3027, 30902, 1428, 985, 1218, 1507, 1428, - -3027, 219, 1509, 1276, 33855, 1225, -3027, 1276, 1450, -3027, - -3027, 52829, 19715, 14047, 66098, 1704, -3027, -3027, -3027, -3027, - 1513, 62162, 1240, -3027, -3027, -3027, -3027, -3027, -3027, 76, - 1750, 204, 1752, 19715, 204, 204, 1250, 237, 237, -3027, - 1441, 1258, -3027, 238, 1262, 1268, 1776, 1778, 168, 151, - 978, 204, 19715, -3027, 237, 1273, 1781, 1278, 1786, 161, - 196, -3027, 239, 19715, 19715, 19715, 1644, 19715, 10428, 1784, - 44957, -3027, 52337, 475, -3027, 386, 1283, 810, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, 1290, -3027, 209, 7183, -3027, - -3027, -3027, -3027, 1326, -3027, -3027, -3027, -3027, 1505, 19715, - -3027, -3027, 1294, 1493, -3027, 241, -3027, -3027, 1493, -3027, - -3027, -3027, -3027, -3027, 270, 1703, 19715, 19715, 62162, 386, - 64130, -3027, -3027, -3027, -3027, -3027, -3027, -3027, 712, -3027, - 323, 43465, 1296, 1301, 834, 52337, 52337, 1774, -3027, -3027, - -3027, 49877, 104, 1596, 1436, -3027, -3027, 1382, 1382, 15062, - 1297, 183, 64, 15579, 20232, 1652, 1534, 675, 600, 1656, - -3027, 1539, 1765, 24368, 19715, 19715, 965, 1006, 19715, 859, - -3027, -3027, -3027, 1586, 52337, 47909, 758, 945, 1314, 1401, - 1319, 62, 1734, -3027, 1320, -3027, 1406, 52337, 68021, 232, - -3027, 1801, 232, 232, 273, 1803, 1415, 285, 1598, 537, - 298, 1320, 2069, -3027, 49877, 234, 576, 1320, 52337, 1443, - 641, 1320, 1764, 62162, 1153, 39893, 1353, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, 165, 14047, -3027, - 1183, 1217, 1242, 379, 121, 1315, 1376, 14047, 1435, 1471, - 179, 1517, 1522, 1527, 1533, 1537, 1567, 1602, 1623, 149, - 1635, 1649, 1653, 1655, 1657, 1661, -3027, 1674, 181, 1685, - 135, 14047, 1687, -3027, 43465, -19, -3027, -3027, 1693, 194, - -3027, 1452, 1459, 62162, 1412, 52337, 741, 1735, 1789, 66575, - 52337, 1612, 2069, 1616, 1381, 1854, 1621, 1064, 1622, 1388, - -3027, 64622, 1894, -3027, -3027, -3027, -3027, -3027, 62162, 44957, - 1393, -3027, -3027, 62162, -3027, 52337, 52337, -3027, 52337, 62162, - -3027, 729, 43465, 1890, 6647, 46433, -3027, -3027, -3027, -3027, - 924, 1105, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, 810, 44957, -3027, 4448, 52829, 42644, 1398, 19715, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, 1399, 1743, -3027, - -3027, 4580, 1403, 42893, 1405, 24368, 24368, 386, 1179, -3027, - -3027, 24368, 1413, 46925, 42556, 1404, 1419, 42961, 16096, 19715, - 16096, 16096, 43028, -3027, 1420, 43226, 57257, 1416, 52337, 51353, - -3027, -3027, -3027, 19715, 19715, 4043, 51845, 1465, 30902, -3027, - 30902, -3027, 1694, 30902, -3027, -3027, 2800, -3027, 30902, 1712, - 19715, 30902, -3027, 30902, 1660, 1662, 1426, 30902, -3027, 52337, - 1431, 52337, -3027, -3027, -3027, -3027, -3027, 43465, -3027, 1430, - 738, 1433, -3027, -3027, -3027, -3027, -3027, 1489, -3027, 1489, - 1489, -3027, -3027, -3027, -3027, 1439, 1439, 1440, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, 1446, 978, -3027, 1489, -3027, 1439, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 68021, -3027, -3027, -3027, -3027, 423, - 467, -3027, 1449, -3027, -3027, 1453, -3027, 1454, 1913, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, 5953, 754, - 1439, -3027, -3027, 1863, -3027, -3027, 19715, 19715, -3027, -3027, - 1455, 43465, 1485, -3027, -3027, 19715, 19715, -3027, -3027, -3027, - -3027, 1954, -3027, 19715, 1489, 1489, -3027, 43902, -3027, 38285, - 16613, 1542, 1546, 1954, -3027, 1954, -3027, 43902, 1959, 1959, - 36315, -3027, 1620, 43277, -3027, 1468, 1180, 7898, 1460, -3027, - -3027, 1461, -3027, 1469, 1463, 40552, 178, 386, 386, 19715, - -3027, 1954, 19715, 7395, 7395, -3027, 207, 66098, 19715, 19715, - 19715, 19715, 19715, 19715, 19715, 19715, 44465, 1551, 167, 62162, - 19715, 19715, 1474, 815, -3027, 19715, 1701, -3027, 1475, 19715, - 1565, 903, 19715, 19715, 19715, 19715, 19715, 19715, 19715, 19715, - 19715, -3027, -3027, 26934, 224, 522, 1811, 1830, -16, 342, - 19715, 1822, 10945, -3027, 1822, -3027, -3027, -3027, -3027, -3027, - 244, -3027, -3027, 1430, 1430, -3027, 62162, -3027, 52337, 309, - 48893, 19715, -3027, -3027, 1480, 1484, 376, 1549, -3027, -3027, - 52337, 36807, 1780, -3027, 739, 1486, -3027, 42517, 1744, 1780, - 1382, -3027, -3027, 25402, 1630, 1782, 1717, -3027, -3027, 1702, - 1707, -3027, 1512, 43614, 20749, 20749, -3027, 1219, 43465, 1350, - -3027, -3027, -3027, -3027, -3027, -3027, 73, -3027, 52337, 54, - 34347, -3027, 1515, 68, -3027, 1755, 1840, 1812, 1652, 600, - 1520, -3027, -3027, 1196, 1525, 65114, 52337, 1813, 1763, 1814, - -86, 66098, -3027, -3027, -3027, -3027, 52337, 62162, 60701, 65606, - 45449, 52337, 44957, -3027, -3027, -3027, -3027, 52337, 185, 52337, - 8165, -3027, -3027, -3027, -3027, 232, -3027, -3027, -3027, -3027, - -3027, 62162, 52337, -3027, -3027, 232, 62162, 52337, 232, -3027, - 1619, 52337, 52337, 52337, 52337, 1628, 52337, 52337, 810, -3027, - -3027, -3027, 21266, -12, -12, 1747, 11979, 148, -3027, 19715, - 19715, 62162, 1706, -3027, -3027, 768, 1756, 88, -3027, 1581, - 52337, 52337, 52337, 52337, 52337, 1488, -3027, -3027, -3027, -3027, - -3027, 1538, -3027, 1540, 1893, -3027, 2069, 1901, 48401, 910, - 2277, 1904, 1585, 1907, 12496, 2020, 1787, -3027, -3027, 1775, - -3027, 89, -3027, 44957, -3027, 101, -3027, 1777, 170, -3027, - 14047, 19715, -3027, -3027, -3027, -3027, -3027, -3027, 1153, 27437, - -3027, 781, -3027, -3027, 2022, 810, 2022, 614, -3027, -3027, - 2022, -3027, 2004, 2022, -3027, -3027, 66098, -3027, 8033, -3027, - 19715, 19715, -3027, 19715, 1896, -3027, 2054, 2054, 66098, 24368, - 24368, 24368, 24368, 24368, 24368, 634, 1273, 24368, 24368, 24368, - 24368, 24368, 24368, 24368, 24368, 24368, 25919, 490, -3027, -3027, - 787, 2031, 19715, 19715, 1906, 1896, 19715, -3027, 66098, 1562, - -3027, 1564, 1571, 19715, -3027, 66098, -3027, 52337, 1572, -20, - -15, -3027, 1566, 1568, -3027, 1493, -3027, 996, 1022, 52337, - 3418, 5521, 6382, -3027, -3027, 19715, 1905, 2800, 2800, 30902, - -3027, 19715, 1575, -3027, -3027, 30902, 1924, -3027, 2800, -3027, - -3027, 34839, 2800, 66098, 794, -3027, 52337, 66098, 800, 19715, - -3027, 14047, 2089, 66098, 2055, 62162, 62162, 2090, 1587, 1588, - 1954, 1675, -3027, 1676, 1677, 1678, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, 66098, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, 1592, 1589, 19715, 19715, 113, -3027, 8137, - 1593, 1594, 6894, -3027, 1597, -3027, 1599, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 1600, -3027, 1595, -3027, 1604, 1624, - 1626, 1610, 1611, 52337, -3027, 21783, -3027, 62162, -3027, -3027, - 19715, 19715, 52337, -3027, 1644, -3027, 1618, 1625, 38785, -3027, - -3027, -3027, 160, 1052, 43663, 342, 7327, 7327, 7327, 43902, - -3027, -3027, -3027, 1634, -3027, 24368, 24368, -3027, 2550, 1125, - 10428, -3027, -3027, 1953, -3027, 1356, -3027, 1627, -3027, -3027, - 3251, -3027, 38285, 43798, 19715, 174, -3027, 19715, 1474, 19715, - 1708, 7327, 7327, 7327, 271, 271, 160, 160, 160, 1052, - 342, -3027, -3027, -3027, 1629, 19715, 44957, -3027, 1632, 1637, - 1976, 1278, 19715, -3027, -3027, 30902, 1465, -19, 1465, 1954, - 7395, -3027, 859, -3027, 859, -3027, 43465, 52337, -3027, -3027, - 1885, 1640, 30902, 1666, 2099, 2082, 62162, -3027, -3027, 1631, - 1822, 1645, -3027, -3027, 1658, 19715, 3833, 1658, -3027, 1780, - -18, 1862, 1119, 1119, 1219, 1865, -3027, -3027, 1709, -3027, - -3027, -3027, 19715, 13013, 1354, -3027, 1360, -3027, -3027, -3027, - -3027, -3027, 1641, -3027, 1915, -3027, 52337, -3027, -3027, 24368, - 2101, 19715, 35331, 2102, 1897, -3027, -3027, -3027, 1738, 1320, - 19715, 1891, -3027, 156, 1659, 2015, 362, 1966, 62162, -3027, - 278, 299, -3027, 176, 2017, 170, 2027, 170, 44957, 44957, - 44957, 802, -3027, -3027, -3027, 810, -3027, 440, 822, -3027, - -3027, -3027, -3027, 1749, 714, 1320, 2069, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, 256, 721, 1320, 1757, -3027, 1758, - -3027, 1759, 784, 1320, -3027, -3027, 1530, 8873, 43465, 544, - 148, 148, 148, 14047, -3027, 1909, 1911, 1673, 43465, 43465, - 1682, -3027, 162, -3027, 62162, -3027, -3027, -3027, 1706, 1789, - 52337, 2069, 1684, 2146, 1064, 1388, -3027, 1836, 770, 596, - -3027, 62162, 52337, 52337, 52337, 61193, -3027, -3027, -3027, 1686, - 1681, -3027, -31, 1910, 1912, 52337, 1726, 52337, 1319, 2160, - 52337, -3027, 838, 17130, 2050, 52337, -3027, -3027, -3027, 62162, - -3027, -3027, 43465, -3027, -3027, 45941, -3027, -3027, -3027, -3027, - -3027, 44957, -3027, 810, -3027, 810, 1928, 62162, 41044, 810, - 41536, 810, 1696, -3027, 43465, 38824, 43465, 1906, -3027, 255, - 2054, 2491, 2491, 2491, 4704, 2041, 182, 1713, 2491, 2491, - 2491, 420, 420, 255, 255, 255, 2054, 490, 774, 46925, - 1718, -3027, 43465, 43465, -3027, -3027, 1705, -3027, -3027, -3027, - -3027, 1719, 1720, -3027, -3027, -3027, -3027, -3027, -3027, 62162, - 1126, 1465, 4, 4, 4, 4, -3027, 52337, 52337, 52337, - 43465, 2154, 2030, -3027, -3027, 2800, 43465, 52337, -3027, 28436, - -3027, 52337, -3027, 2070, -3027, 2155, -3027, 52337, 842, -3027, - -3027, -3027, 848, 1729, 1588, 66098, 853, 863, -3027, 1954, - 130, 1731, 1458, 1464, 731, 1361, -3027, -3027, -3027, 1740, - 43326, 19715, -3027, 2103, -3027, -3027, -3027, 19715, 19715, -3027, - 38285, -3027, -3027, -3027, -3027, 466, 466, 1751, 10428, 43365, - -3027, 2049, 38863, 43465, -3027, 1620, -3027, -3027, 7395, 19715, - 3147, 3524, 19715, 1753, 19715, 2079, -3027, -3027, 1745, -3027, - -3027, 66098, 19715, 1760, 3907, 24368, 24368, 5395, -3027, 5442, - 19715, 10428, -3027, 39932, 1746, 1766, 1747, 17647, -3027, 1964, - 1767, -3027, 1905, 148, 1905, 1768, -3027, -3027, -3027, -3027, - 1863, -3027, 19715, 1914, 62162, 267, 63, 867, -3027, 386, - 36807, 1666, 19715, 197, -3027, -3027, 1770, -3027, 1658, -3027, - -3027, -3027, 1984, -3027, -3027, -3027, 52337, -3027, 1771, -3027, - 34347, 2095, 10428, -3027, 34347, 52337, 52337, 38910, 2127, -3027, - 62162, 62162, 62162, -3027, 62162, 1772, 1783, 318, 1779, 352, - -3027, 2268, 318, 2110, 190, 1319, 285, 2468, 40, -3027, - -3027, -3027, 1850, 52337, -3027, 62162, -3027, -3027, -3027, -3027, - -3027, 45449, -3027, -3027, 37792, 44957, -3027, 44957, 52337, 52337, - 52337, 52337, 52337, 52337, 52337, 52337, 52337, 52337, 1153, 19715, - -3027, 19715, 1791, 1793, 1794, 1747, -3027, -3027, -3027, 146, - -3027, -3027, -3027, -3027, 298, -3027, 162, 1796, -3027, 48401, - 3115, 1585, 2248, 1789, 840, 61670, -3027, 1798, 1797, 869, - 2069, 1800, 2249, -3027, 910, 48401, -3027, -3027, -3027, 2207, - -3027, 551, 177, -3027, 1064, -3027, 3115, 1388, -3027, 3115, - 43465, 62162, 1843, 170, 887, -3027, -3027, -3027, -3027, -3027, - 62162, 1799, -3027, 1799, -3027, -3027, 1799, -3027, -3027, -3027, - -3027, 24368, 2137, 1802, 66098, -3027, -3027, 52337, -3027, -3027, - -3027, 893, 1804, 1905, 52337, 52337, 52337, 52337, -3027, -3027, - -3027, 18164, 19715, 1845, -3027, 1809, 14564, 2126, -3027, 27940, - -3027, 246, 1816, 34839, 62162, -3027, -3027, -3027, -3027, 1954, - -3027, -3027, 62162, -3027, 1815, -3027, 1820, -3027, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, 19715, 43465, -3027, 43465, - -3027, -3027, -3027, -3027, -3027, 7470, -3027, 1817, 1819, 62162, - 19715, -3027, -3027, -3027, 348, 19715, 19715, 2550, -3027, 43979, - 19715, 66098, 899, 2550, 225, 19715, 3980, 4407, 19715, 19715, - 6119, 38949, -3027, 22300, 13530, -3027, 1821, 19715, 39296, 37299, - -3027, 30902, 2030, 1823, 2030, 810, 1825, 43465, 19715, -3027, - -3027, -3027, -3027, 1869, -37, 32870, 2045, -3027, 1838, 62162, - -3027, 1914, 43465, -3027, -3027, 38285, -3027, -3027, -3027, -3027, - -3027, 2280, 1045, 1828, 1831, -3027, 1284, -3027, -3027, 62162, - 1832, -3027, 1834, 318, -3027, 62162, 1874, -3027, 455, 2145, - 91, -3027, 19715, -3027, 2233, 2313, 2268, 1844, 62162, 52337, - 24368, -3027, 737, 191, -3027, 2130, 52337, 1874, 2269, -3027, - -3027, -3027, 352, -3027, 2167, 2080, -3027, 232, -3027, 19715, - 352, 2084, 261, 62162, -3027, -3027, 3461, -3027, 66098, 170, - 170, -3027, 1433, 1852, 1853, 1856, 1864, 1866, 1867, 1868, - 1870, 1871, 1873, -3027, 1876, 1877, 1878, 1879, 1880, 1882, - 1883, 1884, 1446, 1886, -3027, 1889, 1770, 1892, 1900, 1902, - 1916, 1917, 67052, 1919, 1920, 1927, 1929, 1449, 1933, 924, - 1105, -3027, -3027, -3027, -3027, -3027, -3027, 1128, 1936, -3027, - 1875, -3027, -3027, -3027, 1942, -3027, 1957, -3027, -3027, -3027, - -3027, -3027, -3027, 1888, 1899, -3027, -3027, -3027, 148, 1895, - 1925, 1153, 180, 62162, 1939, 1726, 2363, 18681, 876, 2132, - 1908, -3027, 810, 1585, -3027, 48401, 2777, 743, 1912, -3027, - 171, 1726, -3027, 2310, 1585, 1950, 2386, -3027, 2152, 62162, - 1941, -3027, -3027, 45941, 1799, 5039, 24368, 66098, 906, 907, - -3027, 2444, 2104, 2030, -3027, -3027, -3027, -3027, -3027, 1944, - -65, 1945, 9911, 1943, -3027, -3027, -3027, -3027, -3027, -3027, - 43465, 43465, 62162, 2123, -3027, -3027, 1946, 1948, 35823, 2404, - 1952, -3027, 14047, -3027, 2270, -3027, 29424, -3027, 1588, 1960, - 1588, 66098, 1588, -3027, -3027, 43465, 19715, -3027, -3027, 39981, - 2276, 2550, 2550, 43979, 912, -3027, 2550, 19715, 19715, 2550, - 2550, 19715, -3027, 9392, 608, -3027, 914, -3027, 39382, -3027, - 67529, -3027, -3027, 1845, 810, 1845, -3027, -3027, 1956, -3027, - -3027, -3027, 2013, -3027, -3027, 920, 2382, 1914, 19715, -3027, - -3027, 1967, 34347, -3027, -3027, -3027, -3027, 34347, 318, -3027, - 2133, 1874, 1970, -3027, -3027, -3027, -3027, -3027, -3027, 39421, - -3027, 36, 19715, -3027, 184, 4704, -3027, -3027, -3027, -3027, - 1874, 1064, -3027, 52337, 2445, 2332, -3027, -3027, 43465, -3027, - -3027, 1954, 1954, -3027, -3027, 2155, -3027, -3027, 1974, -3027, - -3027, 1128, 394, 37792, 52337, 52337, -3027, -3027, 1978, -3027, - -3027, -3027, -3027, 298, 2364, 925, 910, -3027, 3115, 3115, - 43465, 52337, 2341, 48401, -3027, 2456, 1985, 52337, 1726, 836, - 836, -3027, 2136, -3027, 2144, -3027, -3027, 2458, 300, -3027, - 19198, 52337, -3027, -3027, 31886, -3027, 5039, 942, -3027, -3027, - 2001, 2003, -3027, 1845, 19715, 2006, 19715, -3027, 22817, 2475, - 2005, -3027, 19715, 2071, 26436, -3027, 19715, -3027, 52337, 57257, - -12, 2012, 57257, -3027, -3027, -3027, -3027, -3027, 19715, -3027, - 2550, 2550, 2550, 19715, -3027, 19715, -3027, -3027, -3027, 2215, - 2123, -3027, 2123, 19715, 3115, 386, 3367, 62162, -22, -3027, - 43465, -3027, -3027, -3027, 52337, -3027, 44957, -3027, 318, 388, - 2014, 19715, 39460, 2250, -3027, -3027, 2279, -3027, 2340, -3027, - 2081, 609, 2097, -3027, -3027, -3027, -3027, 1153, 810, 1585, - 1912, 1950, -3027, 2024, 52337, 3115, 910, 551, -3027, -3027, - -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, -3027, - -3027, 3115, 2457, 2244, 2465, 3115, 43465, 1843, 19715, 281, - -3027, 950, 2461, -3027, -3027, 2535, 2123, 2033, 22817, 2036, - -3027, 2038, 62162, 43465, 2183, -3027, -3027, 2039, -3027, -3027, - 148, 19715, -3027, 40063, 2043, 2046, 2496, 1747, 2071, 2071, - -3027, -37, -3027, -3027, 2464, 31886, 2426, 1064, 318, 2060, - 979, -3027, -3027, -3027, -3027, -3027, 2069, -3027, 39546, 2291, - 139, 2281, 2014, 19715, -3027, 2135, -3027, -3027, -3027, 2528, - -3027, -3027, 48401, 2058, 1950, 1912, 1726, 1950, 2284, -3027, - 2285, -3027, 2067, 39811, 62162, 62162, 1585, 31886, 62162, 2068, - 2071, -3027, 2086, -3027, -3027, -3027, 51353, -3027, 2087, 2088, - -3027, -3027, -3027, 19715, 719, -3027, -3027, 2125, 52337, 983, - 34, 2279, 37792, -3027, 44957, 919, 388, 2375, -3027, -3027, - -3027, -3027, 82, 2293, -3027, 2294, -3027, 43465, -3027, 3115, - 48401, -3027, -3027, -3027, -3027, -3027, -3027, 31886, 2461, -3027, - 739, -3027, 1465, -3027, 739, -3027, -3027, -3027, -3027, 810, - -3027, 1404, 23334, 23334, 23334, 2092, 3115, -3027, 1465, -3027, - 2211, 2281, -3027, -3027, -3027, -3027, -3027, 222, 222, 2479, - -3027, 2157, -3027, 1950, 1002, 62162, 1658, -3027, 1658, 24885, - 2238, 163, 42595, 2459, -3027, 2459, 2459, -3027, -3027, -3027, - 36807, -3027, -3027, 2581, -3027, 213, -3027, -3027, -3027, 1585, - 739, -3027, -3027, 2576, -3027, -3027, -3027, -3027, -3027, 157, - -3027, -3027, -3027, 1465, 318, -3027, -3027, -3027, 1465, 1658, - 23851, 2251, -3027, 2317, -3027, -3027, -3027, -3027, -3027, -3027, - -3027 -}; - -/* YYPGOTO[NTERM-NUM]. */ -static const yytype_int16 yypgoto[] = -{ - -3027, -3027, -3027, 1923, 95, -3027, -3027, 90, -3027, 901, - -3027, 80, -632, 447, -3027, 102, 251, 2197, 3688, 119, - -508, -729, -1209, -903, 105, -1040, 19, -3027, -3027, -3027, - -3027, -3027, -544, 142, -3027, -3027, -682, -2513, -633, -3027, - -2897, -2953, -3027, -3027, -781, -2859, -2002, 106, -3027, -3027, - 109, 3, -2019, 112, 821, -3027, -2464, 120, -712, -1070, - -834, -1084, -3027, -180, 122, 1316, -3027, 5, -2130, -2752, - -649, -3027, -754, -3027, -409, -3027, -700, -3027, -836, -708, - -743, -2723, -1054, -3027, 1554, -460, -3027, 476, -3027, -2487, - -3027, -3027, 462, -3027, -1052, -3027, -2063, 51, -687, -2509, - -2495, -2121, -724, 134, -693, 108, -2069, -1049, -3027, 485, - -3027, -680, -3027, -711, -2114, 123, -3027, -3027, 1378, -731, - -3027, 124, 1402, -2058, 22, 24, -3027, -3027, -3027, -3027, - -3027, -688, 419, -1086, -3027, 368, -3027, -3027, -3027, -3027, - -262, 86, -3027, 7, 3114, -30, -42, -3027, -24, -3027, - -3027, -3027, 515, -3027, -3027, 16, 59, 1633, -3027, -935, - -3027, -1348, 775, -3027, 1769, 1795, -2148, -695, -46, -3027, - 552, -1622, -2072, -511, 981, 1601, 1605, 344, -2416, -3027, - -624, -3027, -248, -3027, -3027, 565, 1043, -1520, -1512, -3027, - -2159, -3027, -538, -427, -3027, -3027, -3027, -3027, -3027, -2350, - -2279, -506, 1014, -3027, 1636, -3027, -3027, -3027, -3027, 126, - -1468, 2688, 611, 206, -3027, -3027, -3027, -3027, 71, -3027, - 793, -244, -3027, 2100, -545, -623, 1859, 32, 253, -1750, - 49, 2119, 407, -3027, -3027, 409, -2026, -1370, 364, -321, - 809, -3027, -3027, -1241, -3027, -1863, -1072, -3027, -3027, -789, - 1448, -3027, -3027, -3027, 1742, 2314, -3027, -3027, 2427, 3042, - -3027, -756, 3298, 1093, -913, 1881, -816, 1887, -837, -822, - -796, 1898, 1918, 1921, 1926, 1931, 1932, -1482, 4586, 823, - 2787, -2134, -3027, 927, 25, -3027, -1352, 79, -3027, -3027, - -3027, -3027, -2489, -3027, -493, -3027, -490, -3027, -3027, -3027, - -1614, -3026, -1638, -3027, 3915, 740, -3027, -3027, 346, -3027, - -3027, -3027, -3027, -1471, -3027, 5571, 639, -3027, -1954, -3027, - -3027, -861, -646, -1126, -875, -1180, -1880, -3027, -3027, -3027, - -3027, -3027, -3027, -1437, -1714, -192, 699, -3027, -3027, 791, - -3027, -3027, -3027, -1672, -2038, -3027, -3027, -3027, 705, 1368, - 23, -654, -1569, -3027, 749, -2310, -3027, -3027, 340, -3027, - -483, -991, -2374, 1773, 47, -3027, -489, -2454, -3027, -3027, - -566, -2572, -1016, -713, -3027, 125, -3027, 294, 127, -1635, - -3027, 6, -3027, -461, -3027, -3027, -2514, -3027, 128, 129, - 2161, -3027, -3027, -3027, -418, -3027, -496, -491, -3027, -3027, - 26, -795, 1478, -3027, 131, 575, -3027, 847, -3027, 751, - 132, 87, 1462, 136, 1150, -3027, -3027, -3027, 30, -494, - 303, -3027, 1151, -3027, -3027, 38, 569, 138, -3027, 499, - 140, -3027, -3027, 77, 2693, 141, 10, -2903, 144, -2672, - -1657, -7, -3027, -3027, -3027, -559, -3027, -2450 + 0, 48, 49, 50, 750, 751, 1829, 1776, 2924, 1222, + 1766, 1346, 752, 2586, 2587, 2588, 2230, 1203, 3356, 2231, + 1204, 1205, 2590, 54, 55, 56, 114, 1234, 753, 754, + 527, 59, 2280, 917, 801, 1092, 803, 1093, 1599, 905, + 1237, 1238, 755, 2628, 2292, 3151, 2677, 3152, 2353, 2286, + 1356, 2347, 1870, 1810, 1357, 535, 1884, 2678, 2633, 1871, + 756, 2575, 2926, 3349, 2601, 3489, 2862, 2863, 3346, 3347, + 2578, 2233, 3416, 3417, 2660, 1757, 3411, 2311, 3288, 2237, + 2218, 2864, 2319, 3247, 2972, 2234, 2844, 2312, 3342, 1845, + 2313, 3343, 3100, 2314, 1807, 1833, 2579, 3418, 2238, 1808, + 2574, 2927, 1745, 2315, 3353, 2316, 546, 2848, 757, 2663, + 1302, 529, 518, 519, 1527, 715, 1278, 716, 758, 935, + 1839, 759, 1295, 1821, 65, 1309, 547, 1842, 760, 1830, + 761, 2647, 2648, 762, 763, 1139, 2531, 2152, 764, 741, + 742, 952, 1326, 743, 71, 1850, 765, 1332, 1333, 960, + 73, 870, 1348, 766, 1349, 1350, 969, 74, 2679, 971, + 972, 75, 76, 768, 3183, 2987, 1311, 1846, 2324, 548, + 769, 2809, 2154, 2534, 3338, 79, 3081, 2157, 1164, 3084, + 3302, 2802, 3079, 2535, 3378, 3433, 3082, 2158, 2159, 3303, + 2160, 770, 818, 1467, 771, 772, 1034, 2008, 773, 1748, + 774, 1094, 86, 87, 1007, 88, 3094, 89, 90, 1723, + 1724, 1725, 848, 860, 861, 2150, 1440, 1954, 853, 1168, + 1692, 835, 836, 2276, 878, 1799, 1687, 1688, 2163, 2542, + 1716, 1717, 1177, 1178, 1942, 3317, 1943, 1944, 1433, 1434, + 3192, 1704, 1708, 1709, 2184, 2174, 1695, 2422, 3010, 3011, + 3012, 3013, 3014, 3015, 3016, 1095, 2716, 3203, 1712, 1713, + 1180, 1181, 1182, 1721, 2194, 92, 93, 2135, 2518, 2519, + 807, 3027, 1457, 1726, 2720, 2721, 2722, 3031, 3032, 3033, + 808, 1002, 1003, 1026, 1021, 1447, 1962, 809, 810, 1919, + 1920, 2391, 1028, 1956, 1972, 1973, 2728, 2444, 1528, 2219, + 1529, 1530, 1987, 1531, 1096, 1532, 1558, 1097, 1563, 1534, + 1098, 1099, 1100, 1537, 1101, 1102, 1103, 1104, 1551, 1105, + 1106, 1575, 1989, 1990, 1991, 1992, 1993, 1994, 1995, 1996, + 1997, 1998, 1999, 2000, 2001, 1150, 1727, 1108, 1109, 1110, + 1111, 1112, 1113, 812, 1114, 1115, 1648, 2129, 2517, 3020, + 3200, 3201, 2793, 3069, 3229, 3329, 3448, 3476, 3477, 3503, + 1116, 1117, 1593, 1594, 1595, 2022, 2023, 2024, 2025, 2123, + 1642, 1643, 1118, 2931, 1645, 2045, 3023, 3024, 1151, 1426, + 1586, 1280, 1281, 1542, 1400, 1401, 1407, 1894, 1415, 1419, + 1924, 1925, 1427, 2091, 1119, 2016, 2017, 2461, 1553, 1120, + 1236, 1598, 2788, 2126, 1646, 2085, 1127, 1121, 1128, 1123, + 1582, 1583, 2478, 2760, 2761, 2055, 2191, 1675, 2196, 2197, + 956, 1124, 1125, 1126, 1282, 522, 1543, 3434, 1322, 1156, + 1283, 2081, 775, 106, 776, 96, 777, 1146, 857, 1147, + 1149, 778, 829, 830, 779, 842, 843, 1469, 1662, 1470, + 866, 101, 102, 1191, 831, 849, 781, 3178 }; -/* 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 -2003 +/* YYTABLE[YYPACT[STATE-NUM]] -- What to do in state STATE-NUM. If + positive, shift that token. If negative, reduce the rule whose + number is the opposite. If YYTABLE_NINF, syntax error. */ static const yytype_int16 yytable[] = { - 514, 1206, 730, 56, 766, 61, 78, 66, 1013, 1112, - 96, 1254, 1076, 913, 514, 692, 73, 863, 1782, 52, - 1375, 1871, 64, 703, 65, 2117, 1722, 1303, 1739, 1256, - 88, 707, 511, 1703, 73, 1348, 1713, 1287, 1750, 1043, - 1738, 2084, 2522, 674, 599, 2489, 638, 1709, 1430, 708, - 1349, 1928, 547, 2382, 512, 1635, 1346, 1536, 1995, 1293, - 2462, 1755, 2464, 1691, 514, 514, 1309, 2896, 512, 2479, - 1802, 2384, 2562, 2563, 2564, 1743, 1350, 1550, 1611, 774, - 2219, 2019, 2020, 1614, 2894, 857, 678, 895, 768, 1111, - 2867, 1117, 916, 1121, 2869, 49, 638, 638, 2443, 2515, - 640, 715, 50, 690, 2478, 51, 54, 2908, 2696, 55, - 671, 646, 57, 2575, 2243, -492, -495, 1548, 512, 512, - 58, 890, 59, 62, 63, 76, 2760, 77, 79, 80, - 2765, 85, 86, 2239, -1976, -1976, 87, 2911, 92, 937, - 93, 95, 644, 644, 97, 2137, 2088, 632, -2001, -2001, - 640, 640, 1281, 723, -1850, 2140, -1850, 2363, 2364, 1518, - 2334, 1206, -1990, -1990, 1281, 525, -835, 642, 2370, 1029, - -1240, 1567, 2374, -832, -1240, 1528, 1569, -832, -1237, -1237, - 2743, 864, 2589, -1221, -1841, 3101, -1858, 2582, 1513, 1514, - 1619, 1065, -1981, -1981, -1999, -1999, 1068, 632, 632, -1241, - 682, -835, 2523, 1054, 3387, 1526, 1577, -1238, -1238, -827, - 3037, 1085, 2034, 1025, 642, -492, -495, 1086, 937, 2034, - -1841, -1850, 2578, 3087, 1317, -1858, 3180, 727, 2621, 2623, - 1694, 2626, -1241, 1281, 21, 2660, 1296, -1079, 3020, 3105, - 1579, 937, -794, -807, -822, -1079, 642, 744, 2150, 642, - 1359, 642, 2401, 2889, 2071, -1100, 2596, 101, 2787, 2236, - 758, 3441, 2072, -1100, 698, 1723, 1848, 3264, 1726, 1727, - 937, 1850, -628, 1615, 1296, 642, 700, 704, 705, 937, - 695, 526, 1567, 700, 2912, 3412, 1568, 1569, 3052, -214, - 1654, 681, 2344, 3102, -214, 3072, 1638, 2346, 3291, 1654, - 1206, 2790, 863, 2732, -449, 2734, 1714, 2803, 3103, 3379, - 3048, 3339, 1032, 2524, 2632, 2733, 1705, 1577, 2518, 745, - 3125, 3193, 1087, 896, 1697, 2643, 2021, 1741, 3425, 916, - 3436, 1544, 2143, 1753, 699, 3308, 1655, 3442, 1705, -764, - 2225, 895, 1754, 3, 4, 1655, 1651, 1274, 2081, 2550, - 2347, 1579, 718, 1567, 721, 3364, 725, 1568, 1569, 1567, - 2753, 3304, 610, 660, 3222, 2167, 2744, 1414, 1310, 1055, - 696, 546, 1305, 3423, 3223, 3010, 25, 26, 27, 1728, - 2804, 1964, 3003, 2773, 3005, 2783, 1275, 1616, 1577, 3095, - 1970, 3351, -1975, -1975, 3354, 2526, 609, 1888, 3071, 98, - 3107, 1656, 1934, 2454, -492, -495, 3329, 2307, 922, 1819, - 3096, 2784, 700, 3011, 3448, 746, 2756, 679, 1415, 923, - 1686, 1687, 1579, 2816, 102, 2168, 3208, 2754, 1579, 3443, - 1888, 1848, 1690, 103, 106, 1849, 1850, 2777, 2335, 32, - 641, 2515, 3420, 2515, 2144, 1282, -648, 3340, 99, 2913, - 1657, 2774, 747, 2519, 1652, 1639, 2691, 1282, 3183, 1657, - 2171, 3395, 897, 2794, 2805, 661, 2633, 1026, 104, 1088, - 3049, 1056, -492, -495, -764, 1556, 37, 3290, 1088, 1033, - 1712, 1034, 3380, 1699, 2933, 3426, 2345, 748, 1455, 3265, - 3418, 2348, 3437, 1530, 3413, 2082, 1756, 3104, 3394, 3191, - 611, 1848, 2226, 1931, 3169, 1849, 1850, 1715, 1564, 39, - 1617, 2795, 1658, 2237, 3160, 2890, 3162, 107, 105, 2151, - 42, 1658, 3175, 1729, 2778, 2660, 1282, 2796, 3341, 2525, - 3388, 2526, 2741, 1730, 734, 1305, 43, 749, 1090, 683, - 3224, 3185, 1027, 1792, 3038, 871, 692, 3181, 2918, 2916, - 1757, 2924, 2357, 2358, 2359, 931, 2378, 1637, 701, 2527, - 44, 2712, 2571, 2572, 1276, 1688, 759, 1536, 1091, 862, - 1689, 2904, 3352, 1712, 2745, 3034, 2224, 2035, 3414, 3444, - 3382, 1995, 1966, 3123, 2445, 1507, 2742, 45, -492, -495, - 2779, 73, 2780, 2630, 2277, 2302, 2592, 692, 1700, 3200, - -1079, 642, 859, 2137, 1283, 3415, 697, 2308, 2466, 514, - 1305, 2583, 2492, 1305, 1305, 2654, 1286, 2073, -1100, 2867, - 514, 527, 1630, 2869, 3353, 2099, 2797, -492, -495, -492, - -495, 3082, 1515, 707, 3236, 2798, 1531, 2336, 3028, 1092, - 2675, 638, 2481, 2162, 2341, 2487, 884, 1392, 3172, 3035, - 2100, 708, 638, 3173, 2575, 514, 514, 1518, 1532, 1742, - -1850, 1077, -1850, 512, 1509, 1418, 3146, 1600, 1523, 1422, - 1523, 1531, 1533, 2031, 512, 1277, -1240, 1528, 514, -832, - -1240, 2551, 2375, 1513, 1514, 3289, 2375, 728, 3054, -1221, - -1841, 729, -1858, 1532, 693, 640, 3059, 56, 868, 61, - 78, 66, 1739, 685, 96, -1241, 640, 1535, 1526, 1036, - 73, 1956, 766, 52, 2396, -827, 64, 1804, 65, 3295, - 1562, 514, 1203, 1808, 88, 1249, -1841, -1850, 866, 1030, - 644, -1858, 2283, 1880, 1300, 1301, 863, 2087, -1241, 1895, - 3078, 671, 671, 3296, 671, 1044, 1030, 3019, 1516, 1521, - -822, 2224, 1612, 886, 2928, 2089, 1916, 2962, 1733, 694, - 920, 1061, 1865, 3061, 3062, 1249, 2315, 21, 1977, 1978, - 1035, 1733, 1300, 1301, 709, 1597, 1598, 1599, 1600, 735, - 2112, 1714, 1734, 742, 710, 1022, 743, -214, -214, 49, - 728, 3272, 3362, 726, 729, 1734, 50, 2894, 1812, 51, - 54, 3021, 1805, 55, 857, 2968, 57, 2183, 2970, 2187, - 2972, 1815, 1817, 2167, 58, 642, 59, 62, 63, 76, - 1714, 77, 79, 80, 3085, 85, 86, 1206, 2015, 1206, - 87, 2984, 92, 887, 93, 95, 1061, 1063, 97, 732, - 1705, 1886, 3325, 3326, 1595, 1596, 1597, 1598, 1599, 1600, - 1085, 1706, 733, 3273, 862, 3096, 1086, 3046, 1085, 1556, - 1963, 1544, 514, 514, 1086, 2113, 3209, 2455, 514, 877, - 1544, 514, 514, 2512, 514, 514, 514, 514, 2867, 740, - 73, 3258, 2869, 3259, 1998, 1714, 1064, 2883, 2296, 2884, - 514, 2197, 3099, 2297, 3366, 728, 2342, 514, 1531, 729, - 3192, 2200, 3372, 3178, 2203, 3438, 1737, 1803, 2316, 25, - 26, 27, 1063, 2016, 744, 514, 1249, 1501, 2317, 2272, - 1532, 3210, 638, 754, 1862, 1863, 1864, 1865, 1814, 638, - 3211, 3047, 1402, 937, 1533, 2342, 514, 751, 1409, 1995, - 2544, 1531, 1531, 3100, 512, 1744, 2703, 2043, 692, 2298, - 1399, 512, 755, 2545, 3212, 514, 2055, 3310, 1714, 1098, - 2351, 1087, 1079, 1532, 1532, 1714, 514, 514, 514, 1087, - 514, 514, 32, 1203, 2044, 3165, 640, 1871, 1535, 1099, - 1302, 1085, 1715, 640, 728, 2286, 745, 1086, 1555, 1334, - 1335, 1344, 1860, 1861, 1862, 1863, 1864, 1865, 1262, 1264, - 1267, 1269, 514, 756, 2644, 2645, 2646, 2647, 2196, 37, - 1747, 757, 2417, 2037, 2171, 1642, 1643, 529, 1649, 514, - 514, 1715, 1399, 632, 2289, 1100, 3213, 1705, 1714, 760, - 632, 2075, 1971, 708, 708, 2076, 708, 761, 1708, 3214, - 1425, 762, 39, 2565, 2001, 3268, 2002, 3117, 1712, 1524, - 1525, 2994, 514, 42, 2560, 3373, 514, 514, 2220, 2221, - 2222, 2159, 772, 1567, 2248, 2561, 514, 514, 514, 43, - 2276, 514, 2017, 2487, 2278, 869, 2467, 2280, 2468, 2056, - 530, 870, 1769, 2549, 3374, 902, 1715, 1712, 2195, 2045, - 2553, 1501, 1087, 44, 2046, 871, 2057, 1293, 1577, 2047, - 1030, 2058, 3318, 1395, 1338, 1339, 1560, 45, 2206, 1816, - 1406, 746, 1739, 2213, 937, 3285, 1563, 1088, 3154, 1770, - 1101, 903, 1803, 2490, 872, 1088, 1061, 1362, 937, 3155, - 1889, 1062, 1579, 1566, 1613, 1890, 1567, 1400, 2295, 2059, - 1568, 1569, 2299, 1618, 2670, 2301, 875, 905, 747, 926, - 927, 928, 1712, 2557, 1771, 2498, 878, 902, 1407, 1715, - 1748, 879, 1412, 1740, 2552, 2198, 1715, -572, 880, 1102, - 2201, 1577, -572, 1089, 881, 2136, 2136, 1090, -2003, 1103, - 728, 1115, -1221, 2585, 729, 1090, 1879, 936, 1881, 1882, - 1848, 1104, -504, 903, 1849, 1850, -1972, -1972, 1851, 1852, - 1853, 1249, 1063, 882, 1772, 1579, -504, 1091, 2597, 1408, - 1249, -504, 889, 1294, 2605, 1116, 1295, 1769, 736, 905, - 737, 1548, 1085, 1105, 1628, 1712, 883, 1629, 1086, 1715, - -1973, -1973, 1712, 749, 1249, 3215, 1305, 2340, 3216, 1020, - 1021, 1809, 1023, -572, 1810, 884, 1305, 2048, 1088, 1305, - 1930, 1064, 1501, 1931, 1770, -1974, -1974, 533, 2049, 534, - 863, 1776, -504, 1777, 1779, 921, 1954, 2929, 1780, 1955, - 2432, 918, 1203, 909, 1107, 2183, 2183, 2183, 1092, 929, - 2233, 3434, -504, 2234, 1995, 537, 1092, 924, 862, 3091, - 1401, 2060, -572, 2290, 930, 1712, 2291, 1108, 738, 2328, - 739, -2003, 2329, 2865, 1119, 1203, 2376, 932, 1090, 2377, - 73, 514, 2379, 1024, 2540, 2377, 1110, 2541, -2003, 2399, - 931, 859, 109, -2003, 3421, 524, 3422, -504, -1977, -1977, - 1305, -504, 573, 1087, 2546, 1693, 3397, 2547, 1120, 1772, - -504, -504, 2537, 1028, 2539, 909, -504, 659, 1720, 1045, - 2608, 672, 3409, 1931, 2667, 1039, 1648, 2377, 514, 514, - 2668, -2003, 1401, 1955, 514, 2671, 514, 3447, 2672, 1745, - 1046, 514, 514, 514, 514, 2673, 2655, 1854, 2672, 2748, - 2662, 2903, 2749, 2262, 2234, 2263, 514, 514, 1819, -1978, - -1978, 514, 2463, 514, 2433, 1048, 514, -504, 638, 2922, - 1855, 514, 2923, 514, 514, 2930, 514, 3445, 2931, 1092, - 514, 2985, 3446, 2352, 1955, 2353, 1588, -504, 3118, 3119, - 512, 1955, 2377, 638, 3149, 638, 3156, 1955, 638, 1931, - 1818, 3384, 3166, 638, 1425, 3167, 638, 3199, 638, 2354, - 2234, 2355, 638, 2814, 2534, 512, 2536, 512, -1979, -1979, - 512, 2391, 640, 1049, 3233, 512, 2807, 1955, 512, 2771, - 512, 2770, 3306, 2772, 512, 3307, -504, 1501, 21, 529, - 1856, 2806, 2882, 2815, 1058, -504, 1060, 640, 1051, 640, - 1052, 3254, 640, 3255, -1980, -1980, -504, 640, 2435, 1088, - 640, 3333, 640, 1059, 3334, 3378, 640, 2501, 3307, 514, - 514, 1078, 2692, 2693, 2891, 1080, 1768, 1082, 514, 514, - 1960, 1783, 1083, -2003, 3419, 2436, 514, 3307, 1084, 73, - 2581, 1096, 1957, 514, 632, 1958, 632, 1959, 1252, 632, - -1982, -1982, 530, 1962, 632, -1983, -1983, 632, 1097, 632, - -1984, -1984, 1114, 632, 1118, 2158, -1985, -1985, 1203, 1090, - -1986, -1986, 514, 21, 728, 514, 2679, 1548, 729, 1122, - 1501, 514, 514, 514, 514, 514, 514, 514, 514, 1203, - 1123, 2680, 2682, 514, 514, 1253, 2678, 3065, 514, 1091, - -1987, -1987, 514, 2138, 2139, 514, 514, 514, 514, 514, - 514, 514, 514, 514, 2084, 1257, 514, 2681, 2683, 2684, - 2465, 1259, 2487, 514, 1907, 1249, 1908, 1260, 2119, 1910, - 25, 26, 27, 1270, 1914, -1988, -1988, 1917, 1272, 1918, - -2003, -504, 1273, 1922, 514, 1278, 708, 1595, 1596, 1597, - 1598, 1599, 1600, 2491, 2491, 2437, -1989, -1989, 2183, 1285, - 2438, 1279, 2870, 1288, 2187, 1085, 514, 1903, -1991, -1991, - 1092, 1086, 1326, 1327, 1085, 1036, 1289, 514, 514, 1311, - 1086, 1255, -1992, -1992, 1312, 529, -1993, -1993, -1994, -1994, - -1995, -1995, 1315, 32, -1996, -1996, 1318, 1857, 1858, 1859, - 1373, 1860, 1861, 1862, 1863, 1864, 1865, -1998, -1998, 728, - -1284, 1389, 1048, 729, 1501, 25, 26, 27, -2000, -2000, - -2002, -2002, 1368, 1203, 1386, 1203, 1759, 1760, 1388, 533, - 37, 534, 1396, 2613, -623, -623, 1403, 3029, -627, -627, - 1336, 1337, 1334, 1335, -626, -626, 863, 1404, 530, 1410, - 3145, 1411, 2814, 1417, 1338, 1339, 1421, 537, 3396, 2090, - 3025, 3026, 3398, 39, 1503, 514, 1423, 529, 1504, 1249, - 2015, 1506, 514, 514, 42, -798, 1087, -805, 32, 3405, - 3406, 1515, 3232, 936, 1519, 1087, 1848, 3431, 3432, 45, - 1849, 1850, -1284, -648, 1851, 1852, 1853, 1623, 1624, -649, - 1934, -795, 2247, -796, 1529, 545, -799, 1249, 2439, 1530, - 574, -797, 1540, 1549, 1559, 37, 1203, 2266, 3439, 2440, - 1606, 1561, 648, 1249, 514, 3081, 1608, 1622, 45, 1610, - 530, 1631, 862, 1632, 1636, 1640, 689, 689, 1062, 1641, - 1064, 1692, 1676, 1678, 1680, 1710, 1711, 1717, 39, 1501, - 1712, 1719, 2865, 514, 514, 1718, 514, 1338, 1339, 42, - 1732, 1501, 514, 514, 514, 514, 514, 514, 2764, 2996, - 514, 514, 514, 514, 514, 514, 514, 514, 514, 514, - 1724, 2487, 1731, 1736, 531, 514, 514, 2216, 1746, 514, - 2888, 1501, 106, 2898, 1752, 719, 514, 1761, 1501, 2092, - 2914, 2094, 2921, 899, 1762, 1766, 1784, 1774, 900, 1775, - 1787, 2104, 1788, 45, 1790, 1791, 1793, 1800, 514, 1794, - 2674, 2676, 514, 1811, 514, 533, 1803, 534, 514, 698, - 1837, 1839, 1088, 3232, 1840, 1842, 1501, 1845, 1909, 1876, - 1501, 1088, 514, -573, 1249, 1868, 1501, 1887, -573, 2141, - 536, 1877, 1884, 537, 638, 1905, 1915, 1921, 1302, 532, - 638, 1919, 1926, 1920, 2294, 1929, 1932, 901, 1501, 1509, - 1516, 1521, 1938, 2152, 1966, 3232, 512, 1933, 514, 514, - 1935, 10, 512, 937, 1936, 1999, 1965, 1937, 2204, 2000, - 1567, 2005, 1090, 3057, 2033, 2010, 1855, 2211, 2008, 699, - 2051, 1090, 2011, 2013, 2012, 2040, 2052, 533, 640, 534, - 2054, 2078, 2079, 2085, 640, 2097, 2098, 699, 514, -573, - 2102, 2115, 2205, 514, 514, 3232, 535, 2129, 2128, 2118, - 692, 2212, 536, 3197, 2130, 537, 2127, 1498, 902, 2131, - 2153, 2241, 2242, 2244, 2245, 2246, 1072, 2132, 514, 514, - 2149, 2154, 2157, 514, 21, 632, 2160, 2165, 2231, 2164, - 2166, 632, 700, 2235, 2240, 2252, 1856, 514, -573, 2249, - 514, 2251, 514, 2253, 903, 1305, 2267, 700, 2268, 2270, - 2273, 2274, 2292, 2300, 2275, 1848, 2279, 1873, 514, 1203, - 904, 2330, 1872, 1092, 2337, 514, 2338, 2349, 514, 2350, - 905, 1548, 1092, 2339, 2343, 2361, 2367, 3140, 3066, 3067, - 2369, 2865, 2381, 2388, 2383, 514, 2389, 2390, 2967, 2398, - 2392, 2393, 2394, 2395, 2397, 2405, 2406, 2411, 514, 1098, - 638, 3287, 2408, 2410, 906, 2365, 2412, 1620, 2413, 2409, - 2414, 2368, 2415, 2416, 2434, 514, 514, 638, 2429, 1099, - 2426, 2458, 512, 2450, 2469, 1073, 2472, 2427, 2441, 2474, - 2451, 2475, 514, 2456, 514, 2482, 2480, 3248, 2457, 512, - 2356, 2470, -629, 514, 2483, 2493, 2496, 2494, 2497, 907, - 2500, 2504, 2505, 2506, 640, 2508, 908, 2511, 2514, 2533, - 2510, 1203, 1203, 1203, 2548, 1100, 25, 26, 27, 2535, - 2969, 640, 2554, 2555, 2556, 2568, 2580, 2584, 1846, 1847, - 2566, 1498, 2567, 2569, 1867, 2579, 2595, 2598, 2594, 2602, - 2606, 2599, 2611, 701, 510, 521, 909, 2620, 2628, 543, - 514, 632, 2631, 2651, 543, 2652, 1249, 2638, 629, 692, - 643, 643, 3043, 910, 2634, 647, 543, 655, 632, 2637, - 655, 2639, 2640, 673, 677, 2664, 2663, 677, 2669, 32, - 543, 543, 1776, 2677, 1777, 1779, 2688, 2700, 1501, 1780, - 2710, 1739, 2685, 1857, 1858, 1859, 2711, 1860, 1861, 1862, - 1863, 1864, 1865, 2694, 2729, 2708, 514, 2723, 629, 629, - 1101, 2738, 2714, 3269, 2757, 2761, 37, 2724, 862, 2769, - 2735, 2755, 2731, 2759, 1203, 2802, 2818, 2775, 2897, 2906, - 2782, 673, 911, 2910, 2919, 2776, 38, 1074, 677, 543, - 677, 677, 677, 2885, 2542, 2886, 2887, 2893, 2926, 39, - 2901, 2905, 2902, 2927, 2234, 753, 2932, 2952, 1305, 1102, - 42, 2954, 514, 1305, 2958, 2973, 3009, 3186, 2963, 1103, - 2974, 2977, 2976, 2997, 3016, 3004, 43, 3007, 3018, 3022, - 3023, 1104, 2787, 3024, 3030, 771, 3031, 3032, 2788, 3036, - 3040, 2573, 1498, 3041, 638, 3042, 3050, 3053, 3055, 3056, - 44, 2789, 514, 3060, 863, -1971, -1972, 3074, 2586, -1973, - 1815, 1817, 1490, 1105, 45, 3073, 512, -1974, 1501, -1975, - -1976, -1977, 3075, -1978, -1979, 2790, -1980, 2791, 692, -1982, - -1983, -1984, -1985, -1986, 514, -1987, -1988, -1989, 3076, -1991, - 514, 514, -1992, 3088, 1302, -1993, 3092, 3079, 640, 3077, - 2726, 514, 2618, -1994, 2619, -1995, 2661, 3106, 2624, 3093, - 2627, 692, 514, 3108, 1107, 514, 3110, 514, 2265, -1996, - -1997, 3269, -1998, -1999, 1501, 514, 2015, 3080, 514, 514, - -2000, 3111, -2001, 514, 514, 3250, -2002, 1108, 21, -1238, - 514, 3086, 3114, 3120, 3121, 3124, 3126, 3132, 3128, 689, - 3135, 3134, 3371, 3138, 3139, 514, 1110, 3148, 3141, 3164, - 3144, 3163, 2763, 3168, 2792, 514, 3174, 1960, 3237, 3171, - 3239, 3176, 3247, 2747, 3189, 3188, 73, -1237, 3198, 1957, - 3196, 2577, 1958, 3204, 1959, 514, 3205, 3206, 3221, 936, - 1962, 3219, 1848, 2590, 2591, 2593, 1849, 1850, 1098, 3220, - -2003, -2003, -2003, 3234, 3235, 3241, 3249, 3238, 2604, 3252, - 3242, 2607, 3244, 3251, 3256, 3276, 2612, 3280, 1099, 3049, - 3282, 3298, 3283, 3286, 1203, 3292, 3299, 2868, 1203, 3300, - 1203, 3304, 2793, 3331, 3309, 3311, 1490, 2794, 3313, 3000, - 3316, 3317, 514, 3321, 514, 3323, 3322, 3328, 1566, 3330, - 3332, 1567, 3324, 3337, 3335, 1568, 1569, 1498, 3349, 3350, - 3342, 3348, 1815, 1817, 1100, 2808, 3355, 3356, 3357, 3386, - 3365, 3376, 3389, 3391, 1305, 2809, 1776, 2895, 1777, 1779, - 25, 26, 27, 1780, 3410, 2795, 1577, 3319, 3367, 3369, - 3370, 3387, 3424, -2003, 3407, 3388, 3435, 3429, 2648, 2649, - 2650, 2796, 3440, 2915, 3450, 3449, 2917, 1050, 2817, 2191, - 2531, 2821, 2750, 3263, 514, 3363, 692, 1501, 3327, 3433, - 1579, 2281, 3063, 3346, 3417, 3184, 3385, 3392, 3411, 1702, - 3218, 2538, 2513, 2810, 514, 514, 2909, 3383, 2509, 514, - 3390, 3381, 514, 32, 2786, 2871, 1801, 2576, 1773, 2617, - 1498, 3115, 2892, 2861, 2499, 1393, 1492, 2485, 1673, 1101, - 2156, 1672, 2309, 2310, 2311, 2312, 2313, 2314, 1633, 514, - 2318, 2319, 2320, 2321, 2322, 2323, 2324, 2325, 2326, 2327, - 37, 1394, 3368, 514, 2642, 2573, 2124, 2495, 514, 514, - 3312, 3240, 2155, 514, 1501, 1677, 2661, 1490, 514, 633, - 2797, 514, 514, 2471, 2373, 692, 514, 1249, 1102, 2798, - 514, 730, 3143, 39, 514, 677, -2003, 888, 1103, 677, - 2961, 514, 677, 1360, 42, 874, 2636, 2635, 3359, 543, - 1104, 2666, 2811, -2003, 3190, 2387, 2288, 2758, -2003, 3315, - 43, 3314, 2424, 1345, 2690, 2488, 638, 2448, 2402, 1347, - 2446, 3008, 1896, 2425, 2736, 2702, 3302, 764, 543, 543, - 1351, 867, 1105, 1813, 44, 514, 2380, 1835, 512, 2093, - 1613, 2095, 1856, 514, 2751, 2558, -2003, 716, 45, 0, - 1352, 0, 0, 1353, 1498, 0, 0, 0, 1354, 0, - 0, 0, 514, 1355, 1356, 0, 865, 521, 643, 0, - 640, 1501, 0, 510, 0, 677, 0, 0, 689, 0, - 0, 0, 0, 1107, 629, 0, 0, 0, 892, 892, - 0, 0, 0, 892, 915, 0, 2900, 899, 0, 0, - 1492, 1588, 900, 1305, 0, 2868, 1108, 655, 655, 655, - 0, 0, 0, 0, 3006, 0, 0, 632, 2430, 2431, - 0, 655, 655, 0, 655, 1110, 0, 2812, 0, 0, - 2813, 0, 0, 0, 0, 0, 0, 677, 0, 0, - 0, 0, 0, 543, 0, 0, 0, 0, 0, 0, - 514, 0, 0, 0, 677, 2934, 2935, 2936, 2937, 0, - 0, 901, 0, 0, 0, 0, 0, 0, 677, 0, - 0, 0, 692, 3098, 0, 0, 862, 0, 0, 514, - 1501, 0, 1490, 0, 0, 0, 0, 0, 3097, 1498, - 1251, 0, 0, 0, 0, 514, 0, 1395, -2003, 0, - 0, 1498, 0, 0, 677, 677, 677, 677, 21, 0, - 0, 0, 0, 0, 0, 1249, 0, 0, 0, 0, - 0, 0, 0, 0, 1501, 0, 0, 0, 0, 514, - 0, 1498, 902, 0, 0, 0, 0, 0, 1498, 1712, - 514, 514, 0, 0, 514, 0, 514, 0, 0, -2003, - -2003, -2003, 0, 1860, 1861, 1862, 1863, 1864, 1865, 0, - 0, 1492, 0, 0, 0, 0, 0, 0, 903, 0, - 0, 514, 0, 0, 0, 1490, 1498, 0, 0, 0, - 1498, 3094, 0, 0, 904, 0, 1498, 0, 0, 0, - 0, 0, 0, 0, 905, 514, 0, 0, 0, 0, - 0, 0, 1302, 0, 0, -2003, 0, 3051, 1498, 0, - 0, 0, 1595, 1596, 1597, 1598, 1599, 1600, 0, 1806, - 0, 1807, 0, 0, 0, 0, 2868, 0, 906, 0, - 892, 915, 0, 677, 0, 0, 1367, 0, 0, 0, - 0, 0, 892, 892, 0, 0, 0, 0, 543, 1911, - 25, 26, 27, 0, 629, 3201, 3202, 0, 0, 0, - 0, 629, 0, 514, 0, 0, 0, 514, 0, 0, - 0, 543, 0, 907, 67, 0, 0, 514, 1424, 514, - 908, 514, 0, 3161, 0, 514, 0, 514, 1505, 514, - 0, 0, 67, 0, 0, 631, 0, 0, 0, 638, - 0, 514, 0, 0, 0, 0, 514, 0, 514, 1490, - 67, 0, 0, 32, 0, 936, 514, 0, 1848, 691, - 909, 512, 1849, 1850, 0, 0, 1851, 1852, 1853, 1203, - 0, 3260, 0, 3262, 514, 0, 0, 910, 0, 543, - 0, 0, 0, 2705, 0, 631, 631, 706, 0, 0, - 37, 0, 2861, 640, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 67, 0, - 0, 0, 3294, 0, 0, 0, 1492, 0, 0, 0, - 0, 514, 0, 39, 0, 1625, 0, 1627, 3297, 0, - 0, 514, 3301, 0, 42, 0, 0, 0, 1493, 0, - 0, 0, 543, 543, 514, 0, 911, 0, 677, 0, - 43, 1912, 0, 0, 0, 0, 0, 0, 514, 1566, - 0, 0, 1567, 0, 0, 0, 1568, 1569, 2716, 2717, - 1367, 0, 0, 0, 44, 0, 514, 0, 0, 0, - 0, 677, 1701, 0, 1490, 0, 21, 0, 45, 0, - 638, 0, 0, 0, 677, 0, 1490, 1577, 0, 0, - 514, 0, 0, 0, -2003, 0, 0, 0, 0, 1492, - 0, 677, 512, 0, 0, 677, 514, 3288, 0, 0, - 1749, 0, 0, 0, 0, 2868, 1490, 1203, 0, 0, - 0, 1579, 638, 1490, 0, 0, 0, 0, 0, 0, - 0, 0, 3203, 0, 640, 0, 0, 0, 1498, 0, - 514, 1495, 0, 0, 512, 0, 3393, 0, 0, 0, - 0, 0, 3227, 0, 0, 514, 514, 514, 1855, 0, - 0, 1490, 0, 0, 0, 1490, 0, 0, 0, 0, - 0, 1490, 638, 3408, 0, 0, 640, 0, 0, 0, - 0, 0, 514, 0, 0, 0, 2442, 0, 0, 0, - 0, 0, 0, 1490, 512, 0, 0, 0, 0, 0, - 0, 0, 1493, 0, 0, 0, 0, 0, -1733, 0, - 0, 0, 0, 0, 0, 3267, 0, -2003, 25, 26, - 27, 0, 0, 514, 0, 0, 640, 0, 1856, 0, - 0, 0, 0, 1492, -2003, 0, 0, 0, 0, -2003, - 1763, 2861, 677, 0, 2925, 3293, 0, 677, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1797, 2163, - 0, 0, 0, 0, 0, 764, 0, 0, 1498, 2172, - 764, 2175, 543, 543, 2186, 543, 764, -2003, 3161, 0, - 2190, 32, 2192, 0, 0, 0, 0, 0, 0, 0, - 0, 1098, 0, 0, 0, 2199, 0, 0, 0, 0, - 2202, 0, 1424, 0, 2207, 2208, 2209, 2210, -1733, 2214, - 2215, 1099, 0, 0, 0, 1495, 0, 0, 37, 0, - 0, 0, 936, 0, 1498, 1848, 0, 0, 21, 1849, - 1850, 0, 1588, 1851, 1852, 1853, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2706, 39, 1367, 1367, 0, -1733, 0, 1100, 1367, 0, - 510, 0, 42, 1493, 0, 0, 0, 0, 1492, -1733, - 0, 0, 0, 892, -1733, 543, 1891, 0, 43, -1733, - 1492, 0, 0, 677, 0, 629, 0, 629, -1733, 0, - 629, 0, 0, -1733, 0, 629, 0, 0, 629, 0, - 629, 0, 44, 3045, 629, 0, 543, 0, 543, 0, - 1492, 0, 0, 0, 899, 0, 45, 1492, 0, 900, - 0, 0, 0, 0, 0, -1733, 2810, 1498, 0, -2003, - 0, 67, 0, 0, 0, 1857, 1858, 1859, 0, 1860, - 1861, 1862, 1863, 1864, 1865, -1733, 0, 0, 0, 0, - 0, 0, 1101, 0, 0, 1492, 1369, 0, 0, 1492, - 3261, 0, 0, 0, 0, 1492, 0, 0, 0, 0, - 25, 26, 27, 0, 0, 0, 1495, 0, 901, 67, - 691, 0, 0, 0, 0, 0, 0, 1492, 0, 0, - 544, 0, 0, 1490, -1733, 544, 0, -1733, 0, 0, - 0, 1102, 0, -1733, 0, 0, 0, 544, 0, 0, - 0, 1103, 0, 0, 0, 0, 1994, 1498, 0, 0, - 873, 544, 544, 1104, 0, 0, 0, 2004, 0, 0, - 0, 885, 0, 32, 0, 1855, -2003, 0, 0, 3116, - 0, 0, -1733, 1595, 1596, 1597, 1598, 1599, 1600, 902, - 0, 0, 0, 0, 0, 1105, 0, 0, 0, 0, - 0, 0, 0, 0, 0, -1733, 764, 0, 0, 0, - 37, 0, 0, 0, 0, 0, 0, 0, 1493, 706, - 544, 0, 0, 0, 0, 903, 0, 0, 0, 0, - 0, 0, 0, 0, 1498, 0, 0, 0, 67, 0, - 0, 904, 0, 39, 0, 1856, 1107, 0, 0, 0, - 0, 905, 0, 2091, 42, 677, 0, 677, 0, 0, - 0, 679, 0, 1490, 0, 0, 0, 677, 2107, 1108, - 43, 1979, 0, 0, 0, 0, 0, 0, 1980, 1981, - 1367, 1669, 1982, 1983, 1984, 906, 0, 0, 1110, -1733, - 1369, 0, 0, 0, 44, 0, 0, 0, 0, -1733, - 1806, 0, 0, 0, 0, 677, 1075, 543, 45, 0, - 0, 1493, 0, 0, 0, 0, 0, 0, -1733, 1490, - -1733, -1733, 1701, 543, 0, 0, 0, 0, 0, 0, - 907, 1495, 0, 543, 2173, 543, 2177, 908, 543, 0, - 0, 1498, 0, 0, 543, 1566, 543, 0, 1567, 0, - 0, 0, 1568, 1569, 0, 0, 0, -1733, 764, 543, - -1733, -1733, -1733, 764, 543, 0, 0, 0, 543, 543, - 543, 543, 0, 543, 543, 1498, 0, 909, 0, 0, - 0, 0, 0, 1577, 0, 0, 0, 0, 2230, 0, - -2003, 0, 0, 0, 910, 0, 1496, 677, 677, 677, - 677, 677, 0, 0, 0, 0, 0, 0, 67, 0, - 0, 0, 0, 0, 0, 2255, 0, 1579, 936, 0, - 0, 1848, 1490, 0, 1495, 1849, 1850, 1492, 0, 1851, - 1852, 1853, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 631, 0, 0, 0, 1493, 2987, 0, 631, 0, - 1498, 0, 1857, 1858, 1859, 0, 1860, 1861, 1862, 1863, - 1864, 1865, 0, 911, 0, 0, 0, 0, 2601, 0, - 0, 0, 0, 0, 0, 0, 1367, 1367, 1367, 1367, - 1367, 1367, 2715, 1508, 1367, 1367, 1367, 1367, 1367, 1367, - 1367, 1367, 1367, 1367, 1498, 1520, 0, 0, 0, 0, + 517, 61, 66, 904, 521, 880, 881, 1221, 77, 714, + 879, 103, 1122, 953, 1290, 1187, 517, 60, 1152, 1022, + 521, 1418, 91, 100, 1780, 2044, 1239, 1393, 865, 1921, + 805, 1779, 520, 69, 713, 2166, 84, 1744, 78, 95, + 1791, 1471, 91, 780, 1292, 1345, 967, 1763, 811, 1676, + 1392, 2537, 1391, 1329, 2133, 883, 1389, 1976, 740, 884, + 813, 1750, 767, 1335, 1577, 2620, 2621, 2622, 2440, 851, + 1852, 1754, 2580, 1826, 847, 1678, 51, 1351, 2547, 2068, + 2069, 2960, 517, 517, 844, 2268, 521, 521, 2536, 1784, + 2520, 52, 2522, 1796, 53, 57, 805, 805, 2672, 863, + 58, 1223, 62, 1227, 819, 1231, 2442, 2929, 63, 2958, + 64, 1352, 1652, 2932, 811, 811, 2976, 1655, 1589, 2501, + 67, 1239, 963, 68, 2573, 70, 813, 813, 2973, 72, + 80, 2759, 81, 82, 968, 2634, 83, 94, 889, 1360, + 897, 97, 1004, 98, 99, 2300, 2137, 1025, 2823, 955, + -1888, -511, 2828, -1319, -515, -1888, 1046, 1898, -2028, -2028, + 2186, 1899, 1900, 3377, 2297, 3170, 1898, 2189, -1260, -1241, + 973, 1900, -1260, 2629, 2394, 815, -1257, -1257, -1879, -847, + -852, 1554, 1555, -1879, -852, 1323, -2019, -2019, -855, 532, + 815, -1888, 1402, -1896, 1046, -1261, 2293, -1896, 1567, 2641, + -855, -2037, -2037, -1258, -1258, -1261, 1659, 2083, 2637, -814, + 3155, 815, 855, 2649, 944, -827, -842, 815, 815, 815, + 2642, -2014, -2014, 3104, 2459, 2083, 2723, 3174, 3485, -524, + 1735, -2039, -2039, 871, 1008, 1339, 1656, 1339, 815, 1009, + 1046, 1746, 1046, -524, 1134, 1605, 1585, 1046, -524, 3263, + 1794, -511, 1782, 815, -515, 1176, 3250, 1608, 3087, 1795, + 1239, 1609, 1610, -227, 2684, 2686, 1746, 2689, -227, 967, + 536, 3460, 1352, 1608, 1179, 874, 2199, 2608, 1610, 1898, + -648, 1541, 1764, 1899, 1900, 1767, 1768, 876, 2104, 999, + 2423, 1323, 1618, -466, 536, 1165, 1755, 1141, 1010, -524, + 3514, 1755, 1323, 2432, 3336, 533, 1608, 2656, 1618, 876, + 1609, 1610, 3070, 2274, 3072, 3139, 1314, 2796, 3452, -524, + 1025, 3115, 1005, 872, 1738, 1938, 1620, 2977, 3, 4, + 3381, 2290, 1004, 537, 3509, -2013, -2013, 1138, 2092, 2406, + 554, 1618, 1620, 2404, 2952, 875, 745, 1353, 1352, 1732, + 1657, 1352, 1352, 1769, 1982, 1315, 1938, 537, 821, 2417, + 2418, 2419, 1728, 1729, 2695, 2093, 2706, 3119, -524, 1011, + 3445, 931, 862, 862, -1099, 1620, 3515, -524, 3101, 3293, + 1559, 1661, -1099, 2120, -1120, 2070, 112, 3498, 2837, 3294, + 2576, 2121, -1120, 2407, 2216, 3412, 1569, 3344, 1836, 3171, + 2130, 1680, 1172, 1608, 3195, 1012, 539, 2816, 2161, 3164, + 932, 2105, 1798, 2367, 3172, 3402, 2512, 3113, 3279, 3496, + 3176, 1013, 2836, 876, 2846, 1774, 3138, 1778, 2106, 1864, + 783, 1014, 782, 2107, 1046, 893, 2795, 3486, 2797, 2013, + -511, 3165, 2148, -515, 2395, 2645, 3262, 1455, 2019, 1775, + 2847, 1496, 3102, 1853, 2217, 1292, 1541, 3468, 2819, 3345, + 3521, 1006, 3362, 1291, 1142, 1015, 1143, 2149, 3516, 113, + -207, 2108, 1620, 1658, 2817, 2573, 2275, 2573, 1174, 1324, + 2094, 3116, 1865, 1867, 1740, 2279, 2754, 1291, 980, 2985, + 2096, 3114, 2978, 2162, 3453, 934, 3510, 1770, 1456, -784, + 1756, 1135, 1693, 2192, 1571, 1756, 1577, 1771, 3467, -511, + 1797, 1016, -515, 541, 3455, 542, 2696, 3193, 1017, 2220, + 2723, 3261, 3487, 945, 3446, 2577, -524, 2282, 3239, 3337, + -668, 3413, 1979, 1316, 1753, 1045, 2408, 541, 1898, 962, + 2405, 545, 1899, 1900, 1541, 3499, -2041, -2041, -2041, 2131, + 2200, 3488, 1671, 3447, 856, 3295, 2044, 2015, 1018, 3425, + 3105, 1681, 873, 2969, 1730, 545, 1753, 2981, 784, 2436, + 919, 1753, 877, 2630, 2631, 2084, 1136, 1019, 2211, 1597, + 3251, 714, 1930, 1731, 1679, 1324, 2775, 1783, 2291, 1040, + 2991, 2983, 953, 2503, 902, 902, 1324, 1854, 903, 903, + 3461, 1741, 2337, 1858, 3173, 3245, 941, 2953, 1554, 1555, + 1694, 858, 2609, 1945, 2524, 3414, 2273, 2693, 2193, 3426, + 2545, 3517, 3000, 3150, 3255, 2186, 2109, -511, 3270, 1965, + -515, 2550, 1780, 1567, -784, 967, 2804, 1550, 1031, 3168, + 1325, 2097, 2317, 2375, 2318, 2929, 1020, 1317, 534, 1032, + 2539, 2932, 2098, 1912, 1913, 1914, 1915, 2671, 2652, -1888, + 2738, 1572, 2136, 2362, -1888, 1915, -511, 1564, -511, -515, + -1319, -515, 1564, 1572, 2343, 2368, 902, -1260, -1241, 3242, + 903, -1260, 3095, 1573, 3243, 2634, 2643, -1879, -847, -852, + 3169, 2805, -1879, 2026, 2027, 1573, 2866, 1574, 3216, 1603, + -1888, 2020, -1896, 530, -1261, 2396, -1896, 2012, 1585, 1576, + 3361, 1862, 2401, 2050, -1261, 2051, 996, 1585, 1557, 1239, + 2064, 1239, 1556, 968, 1562, -842, 1653, 2138, 3029, 3159, + 865, 2047, 993, 3368, 1855, 1774, 1343, 1344, 1343, 1344, + -1099, 2066, 966, 3128, 3129, 3086, 1328, 3145, 2433, 2122, + -1120, 3367, 2433, 1638, 1639, 1640, 1641, 2717, 1144, 1775, + 2065, 833, 814, 1541, -227, -227, 1936, 3437, 1172, 2867, + 91, 780, 1641, 1910, 1911, 1912, 1913, 1914, 1915, 1166, + 2454, 865, 517, 1172, 2995, 2376, 521, 2584, 1173, -592, + 3230, 970, 3232, 517, -592, 2377, 3077, 521, 2273, 104, + 1636, 1637, 1638, 1639, 1640, 1641, 1188, 995, 1565, 1566, + 1559, 2216, 883, 1755, 811, 1755, 884, 2879, 1906, 2602, + 3435, 2707, 2708, 2709, 2710, 811, 813, 1746, 517, 517, + 1569, 2402, 2603, 852, 3078, 2958, 3088, 813, 1747, 2246, + 1377, 1378, 867, 2581, 1174, 2185, 2185, 2327, 105, 2249, + 1572, 517, 2252, 2868, 3493, 3035, 1541, 868, 3037, 1174, + 3039, 3051, 977, 834, 3153, -592, 2004, 3121, 2402, 61, + 66, 2570, 1573, 2244, 2513, 3126, 77, 869, 939, 103, + 536, 1167, 3330, 1140, 3331, 60, 844, 844, 1755, 844, + 91, 100, 2086, 1175, 3165, 517, 714, 1597, 1145, 1277, + 517, 69, 885, 1140, 84, 1352, 78, 95, 1175, 2946, + 2929, 2947, 1153, 3308, -592, 1352, 2932, 1441, 1352, 1746, + 3363, 2618, 1438, 2044, -2010, -2010, 2269, 2270, 2271, 1448, + 1749, 1195, 2619, 3511, 1755, 2411, 1696, 1196, 2303, 1785, + 2208, 1788, 2346, 537, 51, 900, 2475, 3298, 1755, 2336, + 1690, 3248, 1277, 2338, 2582, 815, 2340, 1381, 1382, 52, + 3383, 2766, 53, 57, 1335, 966, 1293, 2245, 58, 2840, + 62, 517, 517, 918, 2348, 3235, 63, 517, 64, 929, + 517, 517, 1697, 517, 517, 517, 517, 2255, 67, 1866, + 1541, 68, 2262, 70, 1046, 91, 780, 72, 80, 517, + 81, 82, 1921, 521, 83, 94, 517, 1696, 1352, 97, + 521, 98, 99, 805, 2607, 3424, 3224, 1756, 3427, 1756, + 805, 2187, 2188, 107, 517, 1277, 1526, 3225, -2041, -2041, + -2041, 811, 1910, 1911, 1912, 1913, 1914, 1915, 811, 3357, + 929, 925, 1197, 813, 2247, 517, 1853, 1780, 1698, 2250, + 813, 930, 1011, 1697, 2356, 2525, 2841, 2545, 2526, 2357, + 2611, 1195, 23, 906, 517, 2355, 2548, 1196, 865, 2359, + 958, 1046, 2361, -208, 2615, 517, 517, 517, 3061, 517, + 517, 3187, 714, 1753, 1608, 1753, 937, 2220, 1012, 2556, + 1939, 2623, 1756, 2400, 874, 1940, 3340, 936, 902, 1699, + -1241, 3280, 903, 1195, 1442, 942, 1812, 1591, -593, 1196, + 1198, 517, 930, -593, 1014, 3491, 2358, 2610, 3391, 1618, + 1195, 1572, 2842, 541, 2843, 962, 1196, 2449, 517, 517, + -2011, -2011, 1684, 1685, 1541, 1691, 884, 884, 1756, 884, + 3398, 3399, 946, 1573, 1337, 1813, 1541, 1338, 931, 1444, + 902, 545, 1756, 1620, 903, 1451, 3281, 1574, 1753, 1572, + 943, 1700, 1589, 517, 875, 3282, 967, 517, 517, 948, + 1699, 2583, 1197, 2584, 2457, 949, 1541, 517, 517, 517, + 3160, 1573, 517, 1541, -593, 947, 862, 932, 2657, 3283, + 2092, 108, 1140, 3439, 2665, 1576, 959, 2493, 1198, 817, + 817, 1526, 109, 2585, 1753, 27, 28, 29, 1607, 1812, + 1929, 1608, 1931, 1932, 1197, 1609, 1610, 2093, 1753, 1541, + 2733, 902, 933, 1541, 2494, 1596, 1035, 1036, 1037, 1541, + 1815, 1197, 1700, -593, 1277, 115, 2124, 110, 980, 531, + 2125, 1018, 876, 1277, -2012, -2012, 1618, 744, 1813, 2491, + 921, 1541, 922, -2041, 1781, 1199, 923, 1435, 924, 1200, + 1443, 3284, 832, -2015, -2015, 1449, 845, 1277, 2521, 1454, + 34, 978, 934, 2044, 3285, 1459, 2412, 2523, 2413, 1463, + 1620, 1129, 1130, 1814, 1132, -2016, -2016, 111, 901, 1526, + 979, 1201, 1195, 1195, 998, 2996, 1820, 1822, 1196, 1196, + 981, 1369, 1370, 1823, 984, 714, 1824, 39, 987, 2850, + 2549, 2549, 1669, 1411, 714, 1670, 1859, -2017, -2017, 1860, + 2718, 1195, 2592, 2725, 2594, 989, 1978, 1196, 1198, 1979, + 1847, 966, 2094, 1815, 1548, 3507, 47, 2095, 2002, 988, + 41, 2003, 2096, 2287, 2495, 2490, 2288, 2350, 714, 2496, + 2351, 44, 2853, 2388, 517, 2414, 2389, 2415, 1957, 91, + 780, 1959, 1202, 2595, 2434, 2597, 1963, 2435, 1027, 1966, + 1198, 1377, 1378, 1869, 1970, 2437, -2041, 2559, 2435, 2930, + 970, 3494, 2833, 3495, 2835, 1225, 990, 1198, 2877, 1200, + 3470, 877, 2640, -2041, 2598, 2870, 2604, 2599, -2041, 2605, + 991, 517, 517, 1197, 1197, 47, 3482, 517, 1169, 517, + 1171, 2834, 992, 521, 517, 517, 517, 517, -2018, -2018, + 993, 1226, 2945, 2668, 3520, 2730, 1979, 1229, 2435, 517, + 517, 1200, 1197, 1030, 2731, 517, -2041, 2003, 517, 521, + 1033, 811, 521, 517, 2955, 517, 517, 521, 1200, 805, + 521, 517, 805, 813, 2869, 521, 2878, 805, 2755, 2756, + 805, 3518, 1038, 1230, 1039, 805, 3519, 811, 2734, 2736, + 811, 2735, 2735, 3286, 1040, 811, 3287, 907, 811, 813, + 1201, 862, 813, 811, -2020, -2020, 1041, 813, 1381, 1382, + 813, 1629, 1202, 2097, 1133, 813, 1137, 2744, 2746, 2747, + 2497, -2021, -2021, 908, 2098, 854, 1148, 1589, 1526, 3253, + 2811, 2498, 1154, 2812, 2857, 3326, 1441, 3327, -2022, -2022, + 2743, 2745, 2742, 2968, 2984, 2741, 2288, 2351, 2989, 2997, + 3052, 2990, 2998, 2003, 1202, 1045, 3188, 536, 1898, 2003, + 517, 517, 1899, 1900, -2023, -2023, 1901, 1902, 1903, 517, + 517, 1202, 3189, -2024, -2024, 2435, 815, 517, 1541, 1198, + 1198, 2005, -1305, 2858, 517, 1155, 91, 2011, 909, 1157, + 892, 1158, 895, 1163, 899, 3219, 1159, 2006, 2003, 2859, + 2009, 3226, 2007, 2010, 1979, 1162, 2737, 2739, 1198, 714, + 1189, -2041, 1170, 517, 1192, 2545, 517, 910, -2025, -2025, + 537, 1526, 517, 517, 517, 517, 517, 517, 517, 517, + 714, 2676, -2026, -2026, 517, 517, 2207, 2253, 1190, 517, + 1200, 1200, 3236, 517, 1194, 3237, 517, 517, 517, 517, + 517, 517, 517, 517, 517, 2080, 2133, 517, 3269, 1193, + 911, 2288, 967, 1206, 517, 2260, 1277, -2027, -2027, 1200, + -524, 1011, 1201, 2254, 961, 1856, 2168, 1857, 3132, 2064, + 884, -2029, -2029, 3273, -524, 517, 2351, 3305, 536, -524, + 2003, 1207, 3374, 3379, 3096, 2351, 3380, 14, 15, 1233, + 2860, 2261, 2877, 3406, -2030, -2030, 3407, 1012, 517, 2861, + 1224, 1541, 3451, -1305, 3492, 3380, 1228, 3380, -2041, 517, + 517, -2031, -2031, 1450, 1232, 1636, 1637, 1638, 1639, 1640, + 1641, 1288, 1145, 1014, -2032, -2032, -2033, -2033, -2034, -2034, + -524, 1285, 23, 1202, 1202, 1904, -2036, -2036, -2038, -2038, + 540, 537, 23, 1896, 1897, 1526, 1289, 2827, 1294, 1917, + -524, 1352, -2040, -2040, 714, 1310, 714, 1541, 1905, 1800, + 1801, 1296, 1202, 1297, 1982, 2951, -643, -643, 2979, -647, + -647, 3149, 1312, 2963, 1313, 912, -646, -646, 1320, 2232, + 1321, 2236, 1379, 1380, 2988, 1327, 913, 1381, 1382, 3469, + 541, 1330, 962, 3471, 1331, 2302, 517, 3092, 3093, -524, + 1277, 3478, 3479, 517, 517, 1336, 3063, 3304, -524, 543, + 3504, 3505, 1664, 1665, 1354, 544, 1355, 1358, 545, 1429, + 1361, 914, 1416, 1431, 1432, 3215, 1439, 1445, 1906, 1446, + 1452, 1462, 1453, 1544, 1458, 1464, 1545, 1547, -818, -825, + 1556, 1277, 1560, 47, 915, -668, -669, -815, -816, 3512, + 1018, 1570, -819, 1571, -817, 1581, 714, 1590, 1600, 1602, + 1541, 1647, 1663, 1277, 517, 1649, 1651, 1672, 1673, 1443, + 2545, 916, 966, 1682, 1683, 27, 28, 29, 1677, 1175, + 2321, 3036, 1173, 1718, 2424, 27, 28, 29, 1720, 1526, + 2427, 1722, 1733, 517, 517, 1751, 517, 1752, 1758, 1753, + 1759, 1526, 517, 517, 517, 517, 517, 517, 1760, 1765, + 517, 517, 517, 517, 517, 517, 517, 517, 517, 517, + 1772, 541, 1773, 962, 1777, 517, 517, 2930, 1787, 517, + 112, 1526, 1793, 1803, 1804, 1805, 517, 1809, 1526, 1818, + 34, 1819, 1817, 1008, 1828, 1831, 544, 1832, 1009, 545, + 34, 36, 1834, 3124, 1835, 1541, 1837, -524, 517, 1838, + 3304, 517, 1840, 517, 1841, 521, 975, 517, 817, 1848, + 1849, 521, 3267, 38, 1526, 805, 1853, 39, 1526, 1861, + 517, 805, 1277, 1887, 1526, 1889, 1890, 39, 1958, 1926, + 3133, 3134, 1892, 811, 1895, 1955, 1964, 1937, 1029, 811, + 1918, 1927, 1934, 3304, 1969, 813, 1526, 1010, 920, 3034, + 41, 813, 1967, 927, 1968, 1977, 928, 517, 517, 1352, + 41, 44, 1974, 1131, 1352, 902, 1980, 1985, 1157, 903, + 1550, 44, 1541, 1557, 1986, 1562, 1046, 45, 1907, 1908, + 1909, 1981, 1910, 1911, 1912, 1913, 1914, 1915, 3210, 1983, + 1589, 2015, 1984, 3304, 2014, 2048, 2049, 517, 1608, 2057, + 2054, 46, 517, 517, 2082, 2059, 2060, 2061, 2062, 2089, + 2100, 1865, 1867, 23, 2212, 47, 2103, 2101, 1011, 2127, + 865, 2128, 2134, 3359, 2221, 47, 2224, 517, 517, 2235, + 2146, 2028, 517, 2147, 2151, 2239, 875, 2241, 2029, 2030, + 2164, 2806, 2031, 2032, 2033, 2167, 517, 2176, 2177, 517, + 2248, 517, 3320, 2178, 1012, 2251, 2179, 2181, 2180, 2256, + 2257, 2258, 2259, 2198, 2263, 2264, 2202, 517, 714, 2203, + 1013, 14, 15, 3110, 517, 2213, 2206, 517, 2209, 1541, + 1014, 521, 2214, 2215, 2285, 2289, 876, 2328, 2304, 2294, + 2306, 805, 2307, 2308, 517, 23, 1347, 2330, 521, 2322, + 2323, 2325, 2329, 2332, 2331, 1387, 2333, 517, 805, 811, + 2335, 2352, 2339, 1541, 1015, 2360, 23, 1923, 2390, 1898, + 1922, 813, 2397, 2398, 517, 517, 811, 2399, 2403, 2409, + 2410, 2421, 2930, 2426, 2428, 1960, 2439, 2441, 813, 2446, + 2455, 517, 2447, 517, 2448, 3333, 2450, 2451, 2452, 2453, + 2456, 2463, 517, 2466, 1466, 2464, 27, 28, 29, 2468, + 1016, 1780, 3256, 2471, 2467, 2469, 2472, 1017, 2492, 2470, + 714, 714, 714, 2487, 2369, 2370, 2371, 2372, 2373, 2374, + 2508, 2499, 2378, 2379, 2380, 2381, 2382, 2383, 2384, 2385, + 2386, 2387, 1541, 2473, 2509, 2232, 2232, 2232, 2474, 2484, + 2485, 2514, 2515, 2516, 986, 2527, 2528, 1018, 2530, 517, + 2532, 2533, 2538, 2540, 2541, 1277, 1533, 1865, 1867, -649, + 1601, 34, 2551, 2554, 2555, 2558, 1019, 2807, 2552, 1352, + 1604, 2562, 2564, 2566, 2563, 865, 1541, 967, 27, 28, + 29, 2569, 2568, 2572, 1820, 1822, 2606, 2591, 1654, 2593, + 2627, 1823, 2612, 1526, 1824, 2613, 2614, 1660, 39, 27, + 28, 29, 2626, 2624, 2625, 2639, 2644, 2638, 2655, 2654, + 2662, 517, 2658, 2659, 714, 2666, 536, 2673, 2683, 2691, + 2694, 2697, 2700, 1161, 2701, 2702, 2703, 2714, 966, 2715, + 2726, 41, 3322, 2727, 714, 1020, 2732, 2751, 2763, 1869, + 1961, -1305, 44, 34, 3309, 2773, 3311, 2064, 2740, 1045, + 2748, 2757, 1898, 2774, 2544, 2771, 1899, 1900, 45, 1869, + 1901, 1902, 1903, 2787, 34, 2777, 2786, 23, 2792, 2794, + 1286, 2798, 517, 3319, 2801, 36, 521, 2768, 1541, 537, + 39, 2820, 46, 2824, 2818, 1299, 1301, 1304, 1306, 2822, + 2832, 2838, 2839, 2865, 2845, 2881, 47, 38, 2948, 2488, + 2489, 39, 2949, 2950, 811, 2961, 2971, 2957, 2975, 3321, + 2954, 517, 3324, 41, 2962, 2724, 813, 3444, 2966, 2967, + 2970, 2986, 2288, 3404, 44, 2993, 1526, 2789, 2994, 3025, + 2999, 1533, 3019, 538, 41, 3021, 3030, 3040, 1405, 3076, + 45, 3041, 517, 3043, 3044, 44, 3064, 865, 517, 517, + 3408, 3397, 539, 3071, 3074, 3083, 3085, 3089, 3090, 517, + 3091, 45, 3097, 1856, 46, 3098, 3103, 3107, 3099, 3108, + 517, 3117, 3109, 517, 3120, 517, 3122, 3123, 2808, 3127, + 865, 1208, 1526, 517, -2009, 46, 517, 517, 3156, -2010, + 3392, 517, 517, -2011, -2012, 3143, 3140, 3141, 517, 47, + -2013, 1209, 3161, 3175, -2014, 3177, 3179, 3190, 3142, 540, + 27, 28, 29, 517, 516, 528, -207, -2015, 1868, 1533, + 552, -2016, 1466, 517, 3181, 3146, 552, -2017, -2018, -2020, + 802, 2826, 816, 816, 1352, 2005, -2021, 820, 552, 828, + 91, 2011, 828, 517, -2022, 846, 850, 1210, -2023, 850, + -2024, 2006, 552, 552, 2009, 3147, 2007, 2010, -2025, 541, + 2810, 542, 1905, -2026, 2034, 2035, 2036, -2027, 2037, 2038, + 2039, 2040, 2041, 2042, -2029, 34, -2030, -2031, 543, 3144, + -2032, 3191, 714, -2033, 544, 2928, 714, 545, 714, -2034, + -2035, -2036, -2037, -2038, -2039, -2040, 802, 802, -1258, 3154, + 517, 3198, 517, 3162, 3180, 3202, 3184, 2232, 3194, 846, + 3205, 2933, 39, 2236, 850, 552, 850, 850, 850, 3196, + 3204, 3208, 3209, 3211, 3218, 3214, 3233, 3234, 3238, 3241, + 3244, 3246, 1906, 3258, 1820, 1822, 3259, 2661, -1257, 3432, + 3266, 1823, 1211, 1541, 1824, 41, 1208, 1734, 3268, 3275, + 3276, 3277, 3292, 3290, 3291, 1195, 44, 3306, 3313, 3307, + 1761, 1196, 3316, 3328, 3310, 2959, 1209, 3323, 3348, 1208, + 3314, 3116, 45, 3354, 3352, 3355, 3358, 517, 3370, 3371, + 1526, 1786, 3372, 3364, 3382, 3377, 3384, 3386, 3389, 1209, + 865, 1212, 2980, 3067, 3394, 2982, 46, 517, 517, 3390, + 1438, 1213, 517, 3395, 3396, 517, 3401, 3403, 3405, 2724, + 47, 3410, 1210, 1214, 3422, 3421, 3415, 3438, 3423, 3449, + 3428, 3430, 1607, 3429, 3440, 1608, 3442, 3459, 3443, 1609, + 1610, 3462, 517, 3464, 3480, 1210, 3460, 3483, 1533, 1208, + 3461, 3497, 3502, 3508, 1811, 1215, 517, 3513, 3522, 3523, + 1827, 517, 517, 1160, 3130, 2880, 517, 1526, 2589, 1209, + 1618, 517, 1535, 85, 517, 517, 1197, -2041, 2240, 517, + 1277, 2884, 2143, 517, 2616, 2283, 2635, 517, 904, 865, + 1789, 521, 2680, 85, 517, 976, 804, 3185, 3419, 3490, + 3254, 805, 3458, 2956, 1620, 3465, 3484, 1217, 1743, 3289, + 2571, 85, 2596, 2974, 2849, 1210, 1195, 1211, 3456, 811, + 864, 3463, 1196, 2567, 2934, 3454, 1816, 2814, 2779, 2780, + 1208, 813, 1218, 1851, 2675, 2139, 2341, 1863, 517, 3375, + 1211, 1533, 3335, 2813, 3436, 3400, 517, 3506, 2799, 2557, + 1209, 1220, 1907, 1908, 1909, 2543, 1910, 1911, 1912, 1913, + 1914, 1915, 804, 804, 882, 517, 1212, 1436, 1674, 2500, + 1437, 2205, 1885, 1715, 1526, 85, 1213, 2705, 3441, 2553, + 2173, 3385, 3312, 2204, 2873, 806, 1714, 2529, 1214, 1212, + 2829, 2830, 1719, 3028, 2431, 3213, 1210, 983, 1403, 1213, + -2041, 2699, 1953, 997, 2698, 2729, 3260, 2445, 2928, 1388, + 1211, 1214, 2349, 1390, 1394, 3388, 3387, -2041, 2882, 2482, + 1215, 1395, -2041, 1396, 1397, 1398, 1399, 1197, 2753, 2460, + 2546, 2506, 1198, 2935, 2936, 2937, 2938, 2939, 2940, 2941, + 2942, 2943, 2944, 1215, 2504, 2483, 2765, 1535, 2142, 3075, + 2438, 890, 1946, 0, 517, 714, 0, 0, 0, 1212, + -2041, 0, 0, 2265, 0, 0, 3457, 0, 0, 1213, + 0, 0, 1217, 0, 0, 1533, 2281, 2281, 0, 0, + 1869, 1214, 0, 966, 0, 0, 517, 1526, 0, 1216, + 865, 1211, 0, 1200, 0, 1217, 0, 1218, 0, 0, + 0, 0, 517, 1208, 0, 0, 0, 3167, 0, 0, + 0, 0, 0, 1215, 0, 1629, 1220, 0, 0, 2992, + 1218, 0, 1277, 1209, 0, 1219, 0, 0, 0, 0, + 0, 1526, 0, 0, 0, 1535, 517, 0, 0, 1220, + 1212, 0, 0, 1347, 0, 0, 0, 517, 517, 2354, + 1213, 517, 850, 517, 0, 0, 0, 850, 0, 0, + 850, 0, 1214, 0, 0, 1217, 0, 0, 552, 1210, + 2871, 0, 0, 0, 0, 0, 0, 0, 517, 0, + 2872, 0, 0, 1198, 0, 0, 0, 0, 0, 0, + 1218, 0, 0, 0, 1215, 0, 1202, 1607, 0, 0, + 1608, 0, 517, 0, 1609, 1610, 0, 0, 1536, 1220, + 0, 0, 0, 0, 0, -2041, 0, 0, 0, 1533, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 899, 0, 1490, -2003, 543, 900, 0, 0, 0, 0, - 0, 0, 1546, 0, 0, 0, 677, 0, 0, 0, - -2003, 0, 0, 0, 0, -2003, 629, 0, 0, 0, - 0, 0, 629, 0, 0, 0, 0, 0, 543, 0, - 0, 0, 0, 543, 2486, 0, 0, 1492, 1495, 0, - 0, 0, 2385, 2385, 0, 0, 0, 0, 0, 0, - 1496, 0, 0, -2003, 901, 0, 0, 0, 0, 0, - 1493, 0, 1369, 1369, 0, 0, 0, 0, 1369, 1490, - 0, 0, 1493, 0, 0, 0, 1498, 0, 0, 0, - 0, 706, 706, 0, 706, 0, 0, 0, 0, 0, - 0, 0, 0, 1492, 0, 0, 0, 0, 0, 0, - 0, 0, 1493, 0, 0, 0, 0, 0, 1588, 1493, - 543, 1855, 0, 0, 2421, 0, 0, 0, 0, 543, - 0, 0, 0, 0, 0, 902, 0, 0, 0, 0, - 0, 0, 1497, 0, 0, 0, 0, 0, 0, 0, - 544, 0, 1367, 1367, 0, 0, 0, 1493, 0, 0, - 0, 1493, 0, 0, 0, 0, 0, 1493, 0, 1994, - 0, 903, 0, 0, 0, 0, 0, 0, 0, 544, - 544, 0, 0, 1495, 0, 0, 1490, 904, 0, 1493, - 0, 1856, 0, 0, 0, 1495, 0, 905, 2766, 2767, - 0, 0, 629, 0, 0, 0, 1492, 0, 0, 0, - 0, 0, 0, 0, 543, -2003, 0, 0, 0, 629, - 1490, 1496, 0, 2107, 0, 1495, 2819, 0, 0, 0, - 0, 906, 1495, 0, 0, 0, 0, 0, 0, 0, - 0, 2872, 2873, 2874, 2875, 2876, 2877, 2878, 2879, 2880, - 2881, 1985, 1986, 1987, 0, 1988, 1989, 1990, 1991, 1992, - 1993, 0, 0, 543, 0, 0, 1367, 0, 0, 543, - 1495, 0, 0, 0, 1495, 0, 907, 0, 0, 0, - 1495, 0, 0, 908, 0, 1763, 0, 0, 0, 0, - 0, 0, 0, 0, 544, 1490, 1492, 0, 0, 0, - 0, 0, 1495, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1497, 0, 0, 0, - 0, 0, -2003, 909, 0, 0, 0, 0, 67, 1595, - 1596, 1597, 1598, 1599, 1600, 936, 0, 0, 1848, 1490, - 910, 0, 1849, 1850, 0, 1498, 1851, 1852, 1853, 0, - 0, 1763, 0, 0, 0, 0, 0, 677, 0, 0, - 2126, 0, 0, 2988, 0, 0, 0, 0, 1763, 677, - 677, 677, 0, 1492, 0, 0, 0, 0, 0, 0, - 0, 0, 543, 0, 677, 0, 0, 677, 0, 0, - 0, 0, 677, 0, 0, 0, 764, 0, 1857, 1858, - 1859, 0, 1860, 1861, 1862, 1863, 1864, 1865, 0, 911, - 0, 0, 0, 0, 1763, 1763, 0, 1763, 0, 0, - 0, 0, 0, 1822, 0, 0, 0, 0, 0, 0, - 0, 0, 631, 0, 631, 0, 1496, 631, 0, 0, - 0, 1490, 631, 0, 0, 631, 510, 631, 0, 0, - 0, 631, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2641, 0, 0, 0, - 1823, 0, 0, 0, 677, 677, 677, 1497, 0, 0, - 1492, 0, 3044, 0, 543, 0, 1367, 0, 543, 1493, - 0, 1824, 0, 0, 543, 0, 0, 0, 0, 1387, - 0, 0, 0, 0, 0, 0, 0, 0, 1566, 1825, - 0, 1567, 0, 1826, 1492, 1568, 1569, 1570, 1571, 1572, - 1573, 1574, 1420, 0, 0, 0, 0, 1994, 0, 1496, - 0, 0, 0, 0, 0, 1827, 1575, 67, 1828, 0, - 0, 0, 0, 0, 0, 0, 1577, 0, 1855, 0, - 0, 0, 0, 1578, 1829, 0, 1369, 1369, 1369, 1369, - 1369, 1369, 1367, 1367, 1369, 1369, 1369, 1369, 1369, 1369, - 1369, 1369, 1369, 1369, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 0, 0, 0, 1492, - 1551, 2740, 0, 0, 0, 0, 0, 2107, 0, 0, - 0, 0, 1495, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 677, 0, 0, 2041, 543, 1856, 1493, - 0, 543, 543, 543, 0, 0, 0, 1763, 1701, 1763, - 0, 1797, 936, 1492, 0, 1848, 0, 0, 0, 1849, - 1850, 0, 0, 1851, 1852, 1853, 0, 1830, 0, 0, - 543, 0, 2820, 544, 544, 1831, 0, 0, 0, 0, - 0, 0, 0, 1496, 0, 543, 543, 543, 543, 543, - 543, 543, 543, 543, 543, 1493, 1580, 1832, 0, 0, - 0, 0, 0, 0, 706, 0, 0, 0, 0, 0, - 0, 0, 1497, 1581, 0, 0, 2255, 0, 1582, 0, - 1490, 0, 677, 1833, 0, 0, 0, 0, 0, 0, - 0, 0, 1701, 0, 1370, 0, 0, 0, 0, 0, - 0, 1583, 1584, 0, 0, 0, 0, 0, 1797, 0, - 0, 0, 1495, 0, 0, 1492, 1585, 1763, 0, 0, - 0, 0, 1369, 1369, 0, 0, 3187, 0, 1367, 0, - 0, 0, 0, 0, 543, 0, 0, 0, 0, 0, - 0, 677, 677, 677, 677, 0, 0, 3194, 3195, 0, - 0, 0, 0, 1367, 1586, 0, 1367, 1587, 1493, 0, - 543, 764, 0, 0, 0, 1497, 0, 0, 1495, 2971, - 3207, 1588, 0, 0, 1589, 0, 0, 0, 1496, 0, + 0, 1533, 2141, 2928, 2144, 1618, 0, 0, 2873, 0, + 2242, 0, -2041, 0, 1200, 2153, 1217, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1538, 0, + 0, 1533, 0, 0, 1211, 0, 0, 0, 1533, 1620, + 0, 1218, 517, 714, 1539, 0, 2243, 517, 3112, 3111, + 0, 521, 0, 2190, 0, 0, 0, 517, 0, 517, + 1220, 517, 3271, 3272, 0, 517, 0, 517, 1869, 517, + 0, 0, 0, 0, 1533, 0, 0, 0, 1533, 811, + 1008, 517, 0, 1212, 1533, 1009, 517, 0, 517, 0, + 0, 813, -2041, 1213, 0, 0, 517, 0, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 1214, 1533, 2874, 0, 714, + 0, 0, 0, 0, 517, 0, 0, 1202, 0, 0, + 0, 0, 0, 0, 1535, 0, 0, 0, 951, 552, + 552, 0, 0, 0, 3341, -2041, 0, 1215, 0, 0, + 3332, 0, 3334, 0, 1010, 0, 2295, 2296, 2298, 2299, + 0, 2301, -2041, 1536, 0, 0, 0, -2041, 0, 0, + 0, 0, 0, 517, 0, 0, 0, 0, 0, 974, + 528, 816, 0, 517, 0, 0, 516, 85, 850, 1540, + 0, 0, 3366, 0, 0, 0, 517, 802, 3186, 1217, + 0, 1001, 1001, 0, 0, -2041, 1001, 1024, 3369, 0, + 517, 0, 3373, 1538, 521, 0, 0, 0, 0, 0, + 828, 828, 828, 0, 1218, 1011, 0, 1535, 517, 1539, + 0, 0, 0, 1183, 828, 828, 0, 828, 0, 0, + 0, 0, 811, 1220, 1362, 2875, 815, 0, 2876, 0, + 850, 1536, 0, 517, 813, 0, 552, 521, 0, 0, + 1629, 1012, 0, 0, 0, 0, 0, 850, 0, 517, + 850, 0, 0, 0, 0, -1771, 0, 1013, 2928, 0, + 714, 0, 0, 0, 0, 811, 0, 1014, 2600, 0, + 0, 0, 2416, 0, 0, 0, 0, 813, 0, 0, + 0, 1538, 0, 517, 0, 3341, 0, 521, 0, 1363, + 1364, 0, 0, 0, 850, 1287, 0, 1539, 517, 517, + 517, 1015, 0, 0, 0, 0, 0, 3257, 0, 850, + 850, 850, 850, 1308, 0, 811, 0, 0, 3466, 0, + 2632, 0, 1184, 0, 0, 517, 1319, 813, 3264, 3265, + 1365, 1366, 0, 0, 1367, 1368, 0, 1753, 2646, 0, + -2041, 1535, 0, 0, 1540, 3481, 0, 1016, 0, 0, + 0, 1008, 0, 3278, 1017, -1771, 1009, 0, 0, 1001, + 1024, 0, 850, 0, 0, 1410, 517, 0, 0, 0, + 0, 1001, 1001, 0, 0, 0, 0, 552, 0, 1347, + 0, 85, 864, 802, 0, 0, 0, 2681, 0, 2682, + 802, 0, 0, 2687, 1018, 2690, 0, 0, 0, 0, + 552, 0, -1771, 1533, 0, 0, 0, 1465, 0, 1369, + 1370, 0, 553, 1019, 0, 1010, -1771, 1546, 553, 0, + 0, -1771, 982, 0, 0, 0, -1771, 0, 0, 0, + 553, 0, 1540, 994, 0, -1771, 0, 0, 0, 0, + -1771, 0, 0, 0, 553, 553, 0, -2041, 0, 0, + 1536, 0, 0, 0, 1636, 1637, 1638, 1639, 1640, 1641, + 0, 0, 0, 0, 0, 0, 0, 0, 552, 0, + 0, 0, -1771, 1371, 1372, 1373, 1374, 1375, 1376, 1377, + 1378, 882, 1020, 1379, 1380, 1535, 1011, 1185, 0, 2320, + 0, 0, -1771, 0, 0, 0, 0, 1535, 0, 0, + 1538, 85, 0, 0, 0, 0, 0, 553, 0, 23, + 0, 0, 1045, 0, 1666, 1898, 1539, 1668, 0, 1899, + 1900, 0, 1012, 1901, 1902, 1903, 0, 1535, 0, 0, + 0, 552, 552, 0, 1535, 0, 1533, 0, 1013, 850, + 2769, -1771, 0, 1536, -1771, 0, 0, 0, 1014, 0, + -1771, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1410, 0, 0, 0, 0, 0, 0, 0, 0, + 1535, 850, 1742, 0, 1535, 1872, 0, 0, 0, 0, + 1535, 0, 1015, 0, 850, 0, 0, 0, 3450, 0, + -1771, 0, 1533, 1538, 0, 0, 1381, 1382, 0, 0, + 0, 850, 1535, 1186, 0, 850, 85, 0, 0, 1539, + 1790, 0, 0, -1771, 0, 2850, 0, 2636, 0, 0, + 0, 2851, 1873, 0, 0, 0, 0, 0, 1016, 0, + 2650, 2651, 2653, 0, 2852, 1017, 2925, 0, 0, 804, + 0, 0, 0, 1874, 0, 2664, 804, 0, 2667, 0, + 0, 1540, 27, 28, 29, 0, 2674, 0, 2853, 0, + 2854, 1875, 0, 0, 0, 1876, 1806, 0, 850, 2632, + 0, 852, 0, 0, 850, 1018, 0, 1536, 0, 0, + 0, 1549, 0, 0, 0, 0, 0, 1877, 1844, 0, + 1878, 1383, 1384, 1561, 1019, 1533, 0, 951, 0, -1771, + 0, 0, 951, 0, 552, 552, 1879, 552, 951, -1771, + 0, 0, 0, 0, 0, 1385, 1386, 34, 0, 0, + 1587, 0, 0, 0, 0, 1905, 0, 1538, -1771, 0, + -1771, -1771, 0, 0, 0, 1465, 0, 0, 0, 0, + 2711, 2712, 2713, 1539, 1540, 0, 0, 2855, 0, 0, + 0, 0, 0, 0, 39, 0, 0, 0, 1654, 0, + 0, 0, 0, 1020, 0, 0, 1045, -1771, 0, 1898, + -1771, -1771, -1771, 1899, 1900, 0, 0, 1901, 1902, 1903, + 0, 0, 0, 0, 0, 1410, 1410, 41, 0, 0, + 1533, 1410, 0, 516, 3054, 1906, 0, 0, 44, 1880, + 0, 882, 882, 0, 882, 0, 1001, 1881, 552, 1941, + 0, 0, 0, 0, 45, 2856, 850, 0, 0, 802, + 2857, 1536, 802, 0, 0, 0, 0, 802, 0, 1882, + 802, 0, 3073, 1536, 0, 802, 0, 552, 46, 552, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1496, 0, 0, 0, 0, 2250, 2978, 0, 0, 0, - 0, 0, 0, 0, 67, 1857, 1858, 1859, 0, 1860, - 1861, 1862, 1863, 1864, 1865, 0, 0, 0, 0, 0, - 1496, 0, 0, 0, 0, 1855, 3001, 1496, 629, 0, - 0, 0, 0, 0, 0, 0, 1369, 0, 0, 0, - 0, 0, 3013, 0, 0, 0, 2107, 0, 1493, 0, - 0, 0, 1994, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1496, 1701, 0, 1590, 1496, - 0, 1495, 1763, 544, 544, 1496, 544, 0, 0, 0, - 0, 0, 0, 0, 0, 764, 543, 1367, 1370, 0, - 0, 0, 0, 677, 0, 1856, 0, 1496, 0, 1497, + 0, 0, 47, 0, 3166, 0, 1883, 0, 0, 0, + 0, 0, 0, 1536, 0, 0, 0, 1533, 0, 2858, + 1536, 1538, 0, 0, 23, 0, 0, 2821, 1540, 0, + 0, 0, 0, 1538, 0, 2859, 0, 1539, 0, 1045, + 0, 0, 1898, 0, 0, 0, 1899, 1900, 0, 1539, + 1901, 1902, 1903, 0, 0, 0, 1536, 0, 0, 1535, + 1536, 0, 0, 1538, 0, 0, 1536, 0, 0, 0, + 1538, 0, 0, 0, 0, 0, 0, 0, 0, 1539, + 0, 0, 0, 0, 0, 0, 1539, 0, 1536, 0, + 0, 0, 0, 0, 0, 0, 0, 2043, 0, 0, + 553, 0, 0, 0, 0, 0, 1538, 0, 2053, 0, + 1538, 0, 0, 0, 0, 0, 1538, 0, 2965, 0, + 85, 0, 1539, 0, 1533, 0, 1539, 0, 0, 1905, + 0, 0, 1539, 0, 0, 0, 2860, 0, 1538, 0, + 0, 0, 0, 0, 0, 2861, 0, 951, 0, 0, + 0, 3163, 0, 0, 1539, 1907, 1908, 1909, 1533, 1910, + 1911, 1912, 1913, 1914, 1915, 0, 0, 27, 28, 29, + 1607, 0, 1540, 1608, 1347, 0, 0, 1609, 1610, 0, + 3001, 3002, 3003, 3004, 1540, 0, 0, 0, 0, 0, + 0, 0, 1535, 0, 2140, 0, 850, 0, 850, 1906, + 0, 0, 0, 0, 0, 0, 0, 0, 1618, 850, + 2156, 0, 0, 0, 1540, -2041, 0, 0, 0, 0, + 0, 1540, 1410, 0, 0, 804, 0, 0, 804, 2201, + 0, 0, 34, 804, 0, 0, 804, 1533, 0, 0, + 0, 804, 1620, 0, 0, 0, 0, 850, 1535, 552, + 0, 0, 1905, 0, 0, 3231, 0, 1540, 0, 0, + 0, 1540, 0, 1742, 552, 0, 0, 1540, 0, 39, + 0, 0, 0, 0, 552, 2222, 552, 2226, 0, 552, + 0, 1533, 0, 0, 0, 552, 0, 552, 0, 1540, + 0, 553, 553, 0, 0, 0, 0, 0, 0, 951, + 552, 0, 41, 0, 951, 552, 0, 2778, 0, 552, + 552, 552, 552, 44, 552, 552, 0, 0, 0, 0, + 0, 0, 1906, 0, 2925, 0, 0, 85, 0, 45, + 0, 2284, 3118, 0, 0, 0, 0, 0, -2041, 0, + 850, 850, 850, 850, 1308, 850, 0, 0, 0, 0, + 0, 1535, 0, 46, 0, -2041, 0, 0, 0, 2310, + -2041, 0, 0, 0, 0, 0, 0, 47, 0, 0, + 0, 0, 0, 0, 0, 2334, 0, 0, 0, 0, + 0, 0, 0, 1533, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1536, 0, 0, -2041, 0, + 0, 0, 0, 0, 0, 0, 2090, 0, 553, 1907, + 1908, 1909, 0, 1910, 1911, 1912, 1913, 1914, 1915, 0, + 0, 0, 0, 0, 0, 0, 1410, 1410, 1410, 1410, + 1410, 1410, 0, 0, 1410, 1410, 1410, 1410, 1410, 1410, + 1410, 1410, 1410, 1410, 0, 1538, 1535, 0, 0, 0, + 0, 0, 3360, 1629, 0, 0, 0, 0, 0, 0, + 0, 1539, 0, 0, 552, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 882, 850, 0, 0, 0, + 0, 0, 0, 0, 0, 802, 0, 0, 0, 0, + 0, 802, 1045, 0, 0, 1898, 552, 0, 0, 1899, + 1900, 552, 0, 1901, 1902, 1903, 0, 0, 0, 0, + 2443, 2443, 0, 0, 0, 0, 0, 0, 0, 0, + 3055, 0, 0, 1535, 0, 0, 0, 0, 1536, 0, + 0, 0, 1907, 1908, 1909, 0, 1910, 1911, 1912, 1913, + 1914, 1915, 0, 0, 0, 0, 0, 0, 0, 1430, + 0, 0, 0, -2041, 0, 1607, 0, 0, 1608, 0, + 0, 0, 1609, 1610, 0, 0, 0, 0, 0, 0, + 0, 0, 1461, 0, 0, 0, 0, 0, 1538, 552, + 0, 0, 0, 2479, 1536, 0, 0, 0, 552, 2925, + 0, 0, 0, 1618, 1539, 0, 1540, 0, 0, 0, + -2041, 0, 0, 0, 0, 0, 0, 0, 2305, 0, + 0, 1410, 1410, 0, 0, 0, 0, 85, 0, 0, + 0, 0, 0, 0, 0, 0, 3231, 1620, 2043, 0, + 1535, 0, 0, 3274, 1538, 0, 0, 0, 0, 0, + 1592, 0, 0, 0, 0, 0, 0, 0, 1533, 0, + 1539, 0, 0, 0, 3299, 0, 0, 0, 0, 0, + -2041, 802, 0, 0, 1535, 0, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 552, 0, 0, 0, 0, 802, 0, + 0, 0, 2156, 0, 0, 0, 0, 1536, 0, 0, + 0, 0, 2781, 1607, 0, 0, 1608, 0, 0, 0, + 1609, 1610, 0, 553, 553, 1905, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3339, 0, 0, + 0, 0, 552, -2041, 0, 1410, 0, 0, 552, 1540, + 0, 1618, 0, 0, 0, 0, 0, 1538, -2041, 0, + -2041, 804, 0, 1535, 1806, -2041, 0, 804, 3365, 0, + 0, 0, 0, 1539, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1620, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1906, 0, 0, 0, 0, + 0, 0, 1536, -2041, 0, 1540, 0, 1535, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 3064, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 631, 0, 1493, 0, 0, 0, 631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 936, 0, 0, - 1848, 0, 0, 0, 1849, 1850, 0, 0, 1851, 1852, - 1853, 1495, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1492, 1591, 544, 0, 1592, 1593, - 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 0, 0, - 1763, 0, 0, 0, 0, 1841, 0, 0, 0, 0, - 0, 3377, 2255, 0, 0, 0, 0, 1923, 0, 1927, - 0, 0, 0, 0, 0, 0, 3112, 0, 0, 0, - 0, 0, 0, 1367, 0, 0, 0, 0, 0, 0, - 0, 0, 1493, 0, 1497, 0, 0, 0, 1495, 0, - 0, 0, 0, 0, 0, 0, 1497, 0, 0, 3129, - 0, 0, 0, 0, 1546, 543, 0, 0, 0, 0, - 0, 0, 0, 543, 0, 0, 1493, 0, 0, 0, - 0, 0, 2041, 0, 0, 0, 1497, 0, 0, 0, - 0, 0, 0, 1497, 0, 0, 0, 0, 2452, 0, - 0, 0, 0, 0, 0, 0, 0, 3158, 0, 631, - 0, 0, 1857, 1858, 1859, 0, 1860, 1861, 1862, 1863, - 1864, 1865, 0, 0, 0, 0, 631, 0, 0, 543, - 0, 1497, 0, 0, 543, 1497, 0, 0, 0, 0, - 0, 1497, 1369, 1369, 0, 0, 0, 0, 0, 0, - 0, 1493, 0, 0, 0, 1495, 0, 0, 998, 998, - 543, 0, 0, 1497, 0, 0, 0, 0, 0, 0, - -2003, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 543, 543, 0, 0, 0, 0, 0, 0, 1495, - 1370, 1370, 0, 0, 0, 1493, 1370, 0, 677, 0, - 1701, 0, 0, 0, 543, 0, 0, 1496, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 677, 0, - 0, 3229, 0, 0, 1125, 0, 0, 0, 1244, 0, + 0, 0, 0, 0, 0, 1806, 0, 0, 0, 0, + 0, 850, 0, 0, 0, 0, 0, 0, 0, 0, + 2782, 0, 1538, 1806, 850, 850, 850, 0, 1629, 0, + 0, 0, 0, 0, 0, 0, 0, 552, 1539, 850, + 0, 0, 850, 0, 0, 0, 1308, 0, 0, 1536, + 850, -2041, 0, 0, 0, 0, 951, 0, 0, 0, + 0, 0, 1587, 0, 0, 0, 553, 553, -2041, 553, + 0, 0, 0, -2041, 1806, 1806, 0, 1806, 1540, 1535, + 2090, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 2510, 0, 0, 1538, + 0, 0, 0, 0, 0, 1045, 516, 804, 1898, 0, + 0, -2041, 1899, 1900, 0, 1539, 1901, 1902, 1903, 0, + 0, 0, 0, 0, 804, 0, 2704, 0, -2041, 0, + 0, 0, 0, 0, 850, 850, 850, 0, 0, 0, + 0, 0, 0, 552, 0, 1410, 552, 0, 0, 0, + 0, 0, 552, 0, 0, 0, 1536, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1629, 0, 0, 0, + 553, 0, 0, 1540, 0, 1907, 1908, 1909, 0, 1910, + 1911, 1912, 1913, 1914, 1915, 2043, 0, 0, 0, 0, + 1536, 0, 0, 0, 0, 0, 0, 0, 0, 1971, + 0, 1975, 0, 0, 0, 0, 1538, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1856, 1367, 0, 0, 0, 543, 892, 0, 0, 892, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1495, 0, 0, 0, 1292, 0, - 0, 0, 0, 691, 3013, 0, 0, 0, 544, 0, - 0, 677, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 544, 0, 0, 1493, 0, 0, - 0, 0, 0, 0, 544, 0, 544, 0, 1495, 544, - 0, 677, 0, 1566, 0, 544, 1567, 544, 0, 0, - 1568, 1569, 0, 0, 0, 0, 0, 0, 1369, 0, - 544, 0, 0, 0, 0, 544, 0, 1496, 0, 544, - 544, 544, 544, 0, 544, 544, 0, 0, 0, 3129, - 0, 1577, 0, 2955, 0, 0, 0, 0, -2003, 0, - 1566, 0, 0, 1567, 0, 1316, 0, 1568, 1569, 0, - 0, 1361, 3229, 0, 1363, 0, 0, 1374, 1377, 1382, - 1385, 0, 0, 0, 0, 1579, 0, 0, 0, 0, - 0, 0, 0, 1496, 0, 0, 0, 2980, 1577, 1701, - 0, 0, 0, 0, 0, -2003, 0, 0, 0, 0, - 1495, 764, 764, 0, 3229, 764, 0, 0, 1427, 1244, - 0, 0, 0, 1891, 0, 0, 0, 0, 0, 0, - 0, -1735, 1579, 2083, 0, 543, 0, 0, 0, 1511, - 0, 0, 1546, 0, 0, 0, 0, 1857, 1858, 1859, - 2718, 1860, 1861, 1862, 1863, 1864, 1865, 1701, 1527, 0, - 0, 0, 0, 1497, 3229, 0, 0, 0, 0, 1537, - 1538, 1539, 0, 1543, 1547, 1546, 0, 0, 1370, 0, - 0, -2003, 0, 0, 0, 544, 0, 1369, 0, 0, - 0, 0, 0, 0, 67, 0, 1496, 2719, -2003, 0, - 67, 0, 764, -2003, 0, 1609, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 2107, 0, 2372, - 0, 0, 1427, 1427, 1927, 1319, 2762, 642, -2003, 0, - 0, -1735, 0, 0, 0, 0, 0, 0, 0, 0, - 0, -2003, 0, 0, 0, -2003, 0, 0, 0, 0, - -2003, 0, 0, 0, 0, 1647, 1493, 0, 0, 1663, - 1668, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 998, 998, 0, 0, 0, 0, 0, 0, -1735, 0, - 0, 0, 0, 0, 0, 0, 1496, 0, -2003, 0, - 1320, 1321, -1735, 1497, 0, 0, 1588, -1735, 0, 0, - 0, 544, -1735, 0, 67, 0, 0, 0, 0, 0, - 1551, -1735, 0, 0, 0, 0, -1735, 0, 0, 0, - 0, 0, 0, 1369, 0, 0, 0, 0, 0, 0, - 67, 1322, 1323, 67, 0, 1324, 1325, 0, 0, 0, - 0, 0, 0, 1588, 0, 0, 0, 0, -1735, 1497, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1496, 0, 0, 0, 0, -1735, 1495, - 0, 0, 0, 0, 1370, 1370, 1370, 1370, 1370, 1370, - 1546, 0, 1370, 1370, 1370, 1370, 1370, 1370, 1370, 1370, - 1370, 1370, 0, -2003, 0, 544, 0, 0, 0, 0, - 0, 0, 0, 0, 1244, 0, 0, 0, 0, 0, - 1326, 1327, 0, 1244, 0, 0, 0, -1735, 0, 0, - -1735, 0, 0, 0, 0, 0, -1735, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1244, 0, 0, - -2003, 0, 0, 0, 544, 0, 0, 0, 0, 0, - 2503, 0, 1497, 0, 0, 631, 0, 0, 0, 0, - 0, 0, 0, 0, 0, -1735, 0, 0, 0, 691, - 1496, 0, 0, 0, 1328, 1329, 1330, 1331, 1332, 1333, - 1334, 1335, 0, 0, 1336, 1337, 0, 0, -1735, 0, + 1410, 1410, 1539, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, -2041, 0, 0, 0, 0, + 1538, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 2803, + 1540, 0, 0, 0, 0, 2156, 1539, 0, 0, 0, + 0, 0, 0, 0, 0, 0, -2041, 0, 0, 1536, + 0, 850, 0, 0, 0, 552, 0, 864, 0, 552, + 552, 552, 0, 0, 0, 1806, 1742, 1806, 0, 1844, + 0, 0, 0, -1773, 0, 0, 0, 0, 0, 0, + 1412, 0, 0, 0, 0, 0, 0, 0, 552, 0, + 2883, 0, 0, 1536, 0, 0, 0, 0, 1905, 1538, + 0, 0, 0, 552, 552, 552, 552, 552, 552, 552, + 552, 552, 552, 0, 0, 1539, 0, 0, 0, 0, + 0, 0, 0, 0, 1535, 0, 0, 0, 0, 0, + 0, 0, 0, 1045, 0, 2310, 1898, 1540, 0, 0, + 1899, 1900, 850, 1538, 1901, 1902, 1903, 0, 0, 0, + 0, 0, 1742, -2041, 0, 0, 0, 0, 0, 1539, + 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 1906, 0, + 1844, 1540, 0, -1773, 1413, 0, 0, 0, 0, 0, + 1806, 11, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1410, 0, 0, 0, 1536, 0, 552, 0, 0, + 0, 0, 0, 0, 850, 850, 850, 850, 0, 14, + 15, 0, 0, 0, 0, 0, 1410, 0, 0, 1410, + -1773, 0, 0, 552, 951, 0, 0, 0, 0, 1587, + 0, 0, 3038, 0, -1773, 0, 0, 0, 0, -1773, + 0, 553, 0, 0, -1773, 1538, 0, 0, 0, 0, + 1540, 0, 0, -1773, 23, 0, 553, 0, -1773, 3045, + 0, 1539, 1587, 0, 0, 0, 553, 1711, 553, 0, + 0, 553, 0, 0, 0, 0, 1412, 553, 0, 553, + 0, 85, 0, 0, 0, 0, 0, 85, 0, 3068, + -1773, 802, 553, 0, 1540, 0, 0, 553, 0, 0, + 0, 553, 553, 553, 553, 3080, 553, 553, 0, 2156, + -1773, 0, 0, 2825, 0, 2043, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1742, + 0, 0, 0, 0, 0, 1806, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, -2041, 0, 951, 552, + 1410, 0, 0, 0, 0, 0, 850, 0, 0, -1773, + 0, 0, -1773, 0, 0, 0, 0, 0, -1773, 0, + 1413, 0, 0, 3131, 0, 0, 0, 0, 1907, 1908, + 1909, 0, 1910, 1911, 1912, 1913, 1914, 1915, 0, 0, + 0, 0, 85, 0, 0, 0, 1540, 27, 28, 29, + 0, 0, 0, 0, 0, 0, 0, 0, -1773, 0, + 0, 0, 0, 0, 0, 0, 1906, 0, 0, 85, + 0, 0, 85, 0, 0, 0, 0, 0, 0, 0, + 0, -1773, 0, 0, 0, 0, 0, 0, 0, 1607, + 0, 0, 1608, 0, 0, 0, 1609, 1610, 0, 0, + 0, 3148, 0, 0, 1806, 0, 553, 0, 0, 0, + 1536, 0, 34, 0, 0, 0, 0, 2310, 0, 0, + 0, 0, 1587, 36, 0, 0, 0, 1618, 0, 0, + 0, 0, 0, 3182, -2041, 0, 0, 0, 2430, 852, + 1410, 0, 0, 1975, 0, 38, 0, 0, 0, 39, + 1412, 1412, 0, 0, 0, 0, 1412, 0, 0, -1788, + 1538, 1620, 0, 0, 0, 0, 3199, -1773, 0, 40, + 0, 0, 552, 0, 0, 0, 1539, -1773, 0, 0, + 552, 0, 41, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 44, 0, 0, -1773, 804, -1773, -1773, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 45, + 0, 864, 0, 0, 3228, 0, 0, 0, 0, 0, + 0, 553, 0, 0, 0, 0, 3058, 0, 0, 0, + 1592, 0, 0, 46, 0, -1773, 552, 0, -1773, -1773, + -1773, 552, 0, 0, 1413, 1413, 0, 47, 0, 0, + 1413, 0, 0, 0, 0, 0, 0, -2041, 0, -1788, + 0, 0, 0, 0, 0, 0, 0, 552, 0, 0, + 0, 1107, 1107, 0, -2041, 0, 1907, 1908, 1909, -2041, + 1910, 1911, 1912, 1913, 1914, 1915, 0, 0, 552, 552, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - -2003, 0, 0, 0, 1496, 0, 0, 1595, 1596, 1597, - 1598, 1599, 1600, 0, 1838, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, -1788, 850, 0, 1742, + 0, 1540, 0, 552, 0, 553, 0, -2041, 0, 0, + -1788, 0, 0, 0, 0, -1788, 0, 0, 850, 0, + -1788, 3301, 0, 0, 0, 0, 0, 0, 1235, -1788, + 0, 0, 1279, 1284, -1788, 0, 0, 0, 0, 0, + 0, 1410, 0, 0, 0, 552, 1001, 0, 0, 1001, + 0, 0, 0, 0, 553, 0, 0, 0, 0, 0, + 2561, 0, 1629, 0, 85, 0, -1788, 0, 0, 0, + 0, 0, 0, 0, 3080, 0, 0, 0, 0, 0, + 0, 850, 0, 0, 0, 1334, -1788, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 3246, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1497, 0, 679, 0, 0, -2003, 0, 0, - 0, 0, 0, 0, 1595, 1596, 1597, 1598, 1599, 1600, - 1370, 1370, 0, 544, 0, 0, 0, 0, 0, 0, - 0, 0, -1735, 0, 1382, 0, 1382, 1382, 0, 1496, - 0, 0, -1735, 0, 0, 0, 0, 0, 0, 998, - 998, 0, 0, 0, 0, 1338, 1339, 0, 0, 0, - 0, -1735, 0, -1735, -1735, 0, 998, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1497, - 0, 0, 0, 1496, 0, 0, 0, 0, 0, 0, - 67, 1320, 1321, 0, 0, 0, 0, 0, 0, 0, - -1735, 0, 0, -1735, -1735, -1735, 0, 0, 0, 0, + 0, 0, 1587, 0, 1359, 0, 0, 0, 0, 0, + 1404, 0, 850, 1406, 0, 0, 1417, 1420, 1425, 1428, + 0, 0, 0, 0, 0, 0, 0, 2175, 0, 0, + 0, 0, 0, 0, 0, -1788, 0, 0, -1788, 0, + 0, 0, 0, 0, -1788, 0, 0, 0, 0, 0, + 0, 3199, 0, 0, 0, 0, 0, 1468, 1279, 0, + 0, 0, -2041, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 3301, 2132, 0, 0, 1552, 553, + 0, 0, 0, 0, -1788, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1568, 0, 0, + 0, 1742, 0, 0, 0, 0, 0, -1788, 1578, 1579, + 1580, 0, 1584, 1588, 951, 951, 0, 3301, 951, 0, + 0, 1413, 0, 0, 0, 0, 1941, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 552, 0, + 0, 0, 0, 0, 1650, 0, 0, 0, 0, 85, + 85, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1742, 1468, 1468, 0, 0, 852, 0, 3301, 0, -2041, + 0, 0, 0, 0, 0, 0, 1636, 1637, 1638, 1639, + 1640, 1641, 0, 0, 0, 553, 0, 0, 553, 0, + 0, 0, 0, -1788, 1971, 0, 1689, 0, 0, 0, + 1705, 1710, 0, -1788, 0, 951, 0, 0, 0, 0, + 0, 1107, 1107, 0, 0, 0, 0, 0, 0, 0, + 2156, 0, -1788, 0, -1788, -1788, 0, 85, 0, 85, + 0, 1412, 1412, 1412, 1412, 1412, 1412, 0, 0, 1412, + 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 1412, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 544, 1546, 0, 0, 544, - 0, 0, 1322, 1323, 1370, 1923, 1324, 1325, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1340, - 1341, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, -1788, 0, 0, -1788, -1788, -1788, 1279, 0, 85, + 0, 0, 0, 0, 0, 0, 1279, 0, 0, 0, + 0, 0, 0, 0, 0, 85, 0, 0, 0, 85, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1969, 0, 1342, 1343, 0, 0, 0, 0, 1972, - 0, 0, 0, 0, 0, 1496, 1497, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1566, 0, 0, - 1567, 0, 0, 0, 1568, 1569, 0, 0, 0, 0, - 0, 1326, 1327, 0, 0, 0, 0, 0, 2018, 0, - 1497, 0, 0, 0, 2022, 2023, 2024, 2025, 2026, 2027, - 2028, 2029, 0, 0, 0, 1577, 2038, 2039, 0, 0, - 0, 2050, -2003, 0, 0, 2053, 0, 0, 2061, 2062, - 2063, 2064, 2065, 2066, 2067, 2068, 2069, 0, 544, 2070, - 0, 0, 544, 544, 544, 0, 998, 0, 1244, 1579, - 0, 0, 67, 67, 0, 1328, 1329, 1330, 1331, 1332, - 1333, 1334, 1335, 0, 0, 1336, 1337, 2096, 0, 0, - 0, 544, 0, 0, 0, 1497, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 544, 544, 544, 544, - 544, 544, 544, 544, 544, 544, 0, 0, 0, 0, - 1538, 1539, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2991, 0, 0, 0, 0, 1497, - 0, 0, 0, 0, 0, 0, 0, 0, 67, 0, - 67, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, -2003, 0, 0, 0, 0, + 1279, 0, 1363, 1364, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 553, 0, 0, + 0, 553, 553, 553, 0, 1413, 1413, 1413, 1413, 1413, + 1413, 0, 0, 1413, 1413, 1413, 1413, 1413, 1413, 1413, + 1413, 1413, 1413, 1365, 1366, 0, 0, 1367, 1368, 0, + 553, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 553, 553, 553, 553, 553, + 553, 553, 553, 553, 553, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1888, 0, 0, + 0, 0, 0, 0, 0, 0, 717, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, -2003, 0, 0, 0, 0, -2003, 0, 67, - 0, 0, 0, 0, 0, 1923, 1338, 1339, 0, 0, - 0, 0, 0, 0, 0, 67, 0, 0, 2218, 67, - 0, 0, 1244, 0, 0, 2228, 2229, 0, 0, 0, - 0, 2372, 0, 0, 0, -2003, 0, 0, 0, 0, - 1370, 1370, 0, 0, 0, 0, 0, -42, 0, 0, - 0, 1497, 0, 0, 1496, 0, 0, 0, 0, 0, - 1244, 0, -1750, 0, 0, 0, 0, 0, 0, 1, - 0, 0, 0, 0, 0, 0, 1292, 2282, 0, 2, - 0, 3, 4, 0, 0, 0, 0, 0, 0, 0, - 1588, 0, 0, 0, 5, 0, 0, 0, 0, 6, - 1340, 1341, 0, 0, 0, 0, 2304, 2305, 7, 2306, + 0, 0, 0, 0, 0, 85, 1412, 1412, 0, 0, + 0, 0, 1369, 1370, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 8, 0, 1342, 1343, 0, 0, 0, 0, - 1371, 0, 0, 9, 0, 0, 0, 0, 2332, 2333, - 0, 0, 2096, 67, 1940, 10, 0, 11, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 544, 12, 0, - 0, 0, -1750, 0, 0, 0, 0, 0, 0, 0, - 67, 2360, 0, 0, 13, 14, 0, 2366, 0, 0, - 0, 0, 0, 0, 15, 0, 0, -2003, 0, 0, - 16, 0, 0, 0, 0, 1427, 0, 1244, 17, 0, - 18, 19, 0, 0, 0, 0, 0, 0, 0, -1750, - 0, 0, 0, 0, 20, 0, 0, 0, 21, 0, - 0, 0, 0, -1750, 0, 0, 1370, 0, -1750, 0, - 0, 0, 2400, -1750, 0, 0, 0, 0, 0, 0, - 0, 0, -1750, 0, 22, 0, 0, -1750, 0, 1603, - 0, 1370, 0, 0, 0, 0, 0, 0, 0, 0, - 23, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 2419, 0, 0, 0, 0, 2422, 2423, 0, -1750, - 0, 0, 0, 0, 0, 0, 24, 0, 0, 0, - 0, 0, 1603, 0, -2003, 0, 0, 0, 0, -1750, - 0, 1595, 1596, 1597, 1598, 1599, 1600, 0, 0, 0, - 1497, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2444, 0, 0, 2447, 1371, 2449, 3137, 0, 0, 0, - 0, 0, 0, 0, 2372, 0, 0, 0, 0, 0, - 0, 2453, 0, 0, 0, 0, 0, 0, -1750, 1, - 0, -1750, 0, 0, 0, 0, 0, -1750, 0, 2, - 25, 26, 27, 0, 0, 0, 0, 0, 28, 0, - 0, 29, 0, 0, 0, 0, 1603, 0, 0, 6, - 0, 1647, 0, 0, 0, 1370, 0, 0, 7, 0, - 544, 0, 0, 0, 0, 544, -1750, 0, 1668, 2068, - 30, 0, 8, 0, 0, 0, 0, 0, 0, 31, - 0, 0, 0, 9, 0, 0, 0, 998, 0, -1750, - 0, 544, 0, 32, 0, 10, 2507, 11, 0, 0, - 33, 0, 0, 0, 34, 0, 0, 0, 12, 0, - 0, 0, 544, 544, 35, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 13, 0, 36, 0, 0, 0, - 37, 0, 0, 0, 15, 544, 0, 0, 0, 0, - 16, 0, 0, 0, 0, 679, 0, 0, 17, 0, - 38, 0, 0, 1539, 0, 0, 0, 0, 0, 1244, - 0, 0, 0, 39, 20, 1603, 40, 0, 21, 41, - 0, 0, 0, -1750, 42, 0, 544, 0, 0, 0, - 0, 0, 0, -1750, 0, 0, 0, 0, 0, 0, - 43, 1370, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, -1750, 0, -1750, -1750, 0, 0, 0, 2610, - 23, 0, 0, 1603, 44, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 45, 0, - 0, -42, 0, 0, 0, 0, 24, 1603, 0, 0, - 0, -1750, 0, 0, -1750, -1750, -1750, 0, 0, 0, - 0, 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, - 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, + 0, 0, 85, 0, 0, 0, 0, 1425, 0, 1425, + 1425, 0, 0, 718, 0, 0, 0, 0, 0, 1971, + 0, 0, 1107, 1107, 0, 0, 0, 0, 0, 719, + 0, 0, 0, 0, 0, 0, 0, 0, 1107, 0, + 0, 0, 0, 0, 0, 2430, 1371, 1372, 1373, 1374, + 1375, 1376, 1377, 1378, 0, 0, 1379, 1380, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1575, 0, 1603, 0, 1603, 0, 1371, 1371, 0, 1866, - 1577, 0, 1371, 0, 0, 1603, 0, 1578, 1603, 0, - 0, 0, 0, 1603, 0, 0, 1603, 0, 0, 0, - 25, 26, 27, 0, 0, 0, 0, 0, 28, 0, - 0, 29, 0, 0, 1579, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 2687, 0, 0, - 0, 0, 0, 2689, 1972, 0, 0, 0, 1603, 0, - 30, 0, 0, 0, 2695, 0, 0, 0, 0, 31, - 0, 0, 0, 0, 0, 2704, 544, 0, 2707, 0, - 2709, 0, 0, 32, 0, 0, 0, 0, 2713, 0, - 33, 0, 0, 0, 34, 0, 2720, 2721, 0, 0, - 0, 0, 0, 2728, 35, 0, 0, 0, 0, 1370, - 0, 0, 0, 0, 0, 0, 36, 0, 2737, 0, - 37, 0, 0, 0, 0, 0, 0, 0, 2752, 0, - 1580, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 38, 0, 1603, 0, 0, 0, 0, 1581, 998, 0, - 0, 0, 1582, 39, 0, 0, 0, 0, 1603, 41, - 0, 0, 0, 0, 42, 0, 0, 0, 1603, 1603, - 1603, 0, 0, 0, 1603, 1583, 1584, 0, 1603, 0, - 43, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1585, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 44, 2218, 0, 2218, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 45, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1586, 0, - 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, - 1603, 0, 0, 0, 0, 1588, 1565, 0, 1589, 0, - 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1603, 1575, - 0, 0, 0, 1576, 0, 0, 0, 0, 0, 1577, - 0, 0, 0, 0, 1603, 0, 1578, 2950, 2951, 1603, + 1413, 1413, 0, 0, 0, 0, 0, 0, 720, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 721, 0, + 1412, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 722, 0, 0, 0, 0, 723, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1866, 0, 0, 0, - 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, - 0, 0, 2975, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1590, 0, 0, 0, 2979, 0, 0, 0, - 0, 2981, 2982, 0, 0, 0, 2983, 0, 0, 0, - 0, 2986, 0, 0, 2989, 2990, 0, 0, 0, 2218, - 1244, 0, 0, 2998, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 998, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1566, 0, 0, 1567, 0, - 0, 0, 1568, 1569, 0, 0, -2003, -2003, -2003, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, - 0, 0, 0, 0, 0, 0, 0, 0, 3039, 0, - 0, 0, 0, 1577, 0, 0, 1581, 0, 0, 0, - 1578, 1582, 0, 0, 0, 0, 0, 0, 0, 1591, - 0, 0, 1592, 1593, 1594, 3058, 1595, 1596, 1597, 1598, - 1599, 1600, 0, 0, 1583, 1584, 0, 1579, 0, 1603, - 0, 0, 0, 0, 0, 2407, 0, 1866, 1866, 1585, - 1371, 1371, 1371, 1371, 1371, 1371, 0, 0, 1371, 1371, - 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1371, 1866, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1586, 0, 0, - 1587, 0, 0, 1320, 1321, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1588, 0, 0, 1589, 1566, 0, - 0, 1567, 0, 3090, 0, 1568, 1569, 1570, 1571, 1572, - 1573, 1574, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1580, 1322, 1323, 1575, 0, 1324, 1325, - 2699, 0, 0, 0, 0, 0, 1577, 0, 0, 0, - 1581, 0, 0, 1578, 0, 1582, 0, 0, 1547, 0, + 0, 0, 0, 0, 724, 0, 0, 0, 0, 0, + 0, 0, 2018, 0, 0, 0, 0, 0, 0, 0, + 2021, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1381, + 1382, 553, 0, 0, 0, 0, 0, 725, 0, 0, + 0, 726, 0, 0, 1413, 0, 0, 0, 0, 2067, + 0, 0, 0, 0, 0, 2071, 2072, 2073, 2074, 2075, + 2076, 2077, 2078, 0, 0, 0, 0, 2087, 2088, 0, + 0, 0, 2099, 0, 0, 0, 2102, 0, 0, 2110, + 2111, 2112, 2113, 2114, 2115, 2116, 2117, 2118, 0, 0, + 2119, 0, 0, 0, 0, 1607, 0, 1107, 1608, 1279, + 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, + 0, 727, 0, 0, 0, 0, 0, 0, 2145, 0, + 0, 0, 0, 1616, 1383, 1384, 728, 0, 0, 0, + 0, 0, 0, 1618, 0, 0, 0, 0, 0, 0, + 1619, 0, 0, 0, 0, 0, 0, 0, 1385, 1386, + 0, 0, 1579, 1580, 0, 0, 0, 0, 0, 729, + 0, 0, 730, 0, 0, 0, 0, 1620, 1988, 0, + 0, 0, 1607, 731, 0, 1608, 732, 0, 0, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, + 0, 0, 0, 0, 733, 0, 0, 0, 0, 0, + 1616, 0, 0, 0, 3207, 0, 0, 0, 734, 0, + 1618, 0, 2430, 1414, 0, 736, 0, 1619, 0, 0, + 0, 0, 0, 0, 0, 737, 0, 0, 0, 0, + 0, 738, 0, 0, 0, 0, 0, 0, 0, 2267, + 0, 0, 0, 1279, 1620, 0, 2277, 2278, 0, 0, + 0, 0, 0, 0, 0, 1412, 1412, 0, 739, 0, + 0, 0, 0, 1621, 0, 0, 0, 0, 553, 0, + 0, 0, 0, 553, 0, 0, 0, 0, 0, 0, + 1622, 0, 0, 0, 1279, 1623, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 553, + 0, 0, 0, 0, 0, 0, 1334, 2342, 1624, 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1603, 0, 0, 1603, 0, 0, 0, 0, 1244, 0, - 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 2419, 1585, 0, 0, 0, 0, 0, 0, - 0, 1590, 0, 3150, 3151, 0, 0, 3152, 0, 1539, - 0, 0, 0, 1326, 1327, 0, 0, 0, 0, 1603, - 0, 0, 0, 1603, 1603, 1603, 1603, 1603, 1603, 1603, - 1603, 0, 0, 0, 3170, 0, 1371, 1371, 0, 1603, - 1603, 0, 0, 0, 0, 0, 0, 0, 1588, 0, - 0, 1603, 0, 0, 1603, 0, 0, 0, 3182, 0, - 0, 0, 1603, 1603, 1603, 1603, 1603, 1603, 1603, 1603, - 1603, 1603, 0, 0, 0, 0, 1580, 1328, 1329, 1330, - 1331, 1332, 1333, 1334, 1335, 0, 0, 1336, 1337, 0, - 0, 0, 0, 1581, 0, 0, 0, 1603, 1582, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 1583, 1584, 0, 0, 0, 3226, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1585, 0, 0, 0, - 0, 0, 0, 0, 2950, 1590, 0, 0, 3243, 0, - 1371, 0, 998, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3253, 0, 0, 0, 0, 2218, - 0, 2218, 0, 0, 1586, 0, 0, 1587, 0, 998, + 553, 553, 0, 1626, 0, 0, 0, 0, 0, 0, + 0, 0, 1644, 0, 0, 0, 2364, 2365, 0, 2366, + 1621, 0, 0, 0, 0, 553, 0, 0, 0, 1413, + 1413, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 0, 1627, 1623, 0, 1628, 0, 0, 0, 2392, 2393, + -45, 0, 2145, 0, 0, 1644, 0, 0, 1629, 0, + 0, 1630, 0, 0, 0, 1624, 1625, 553, 0, 0, + 0, 0, 1, 0, 0, 0, 0, 0, 0, 0, + 1626, 2420, 2, 0, 3, 4, 2425, 0, 0, 1414, + 0, 0, 0, 0, 0, 0, 0, 5, 0, 0, + 0, 0, 6, 1468, 0, 1279, 0, 0, 0, 0, + 0, 7, 0, 0, 0, 0, 0, 0, 1627, 0, + 0, 1628, 0, 0, 0, 8, 1412, 0, 0, 0, + 0, 0, 0, 0, 9, 1629, 10, 0, 1630, 0, + 1644, 2458, 0, 0, 0, 0, 0, 0, 11, 0, + 12, 3022, 0, 0, 0, 0, 0, 0, 1631, 0, + 0, 13, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 14, 15, 16, 0, + 2477, 0, 0, 0, 1644, 2480, 2481, 17, 0, 1644, + 0, 0, 0, 18, 0, 3047, 0, 0, 0, 0, + 0, 19, 0, 20, 21, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 22, 0, 0, + 1413, 23, 0, 0, 0, 0, 0, 0, 0, 2502, + 0, 0, 2505, 0, 2507, 1631, 0, 0, 0, 1644, + 0, 0, 0, 0, 0, 1413, 0, 24, 0, 0, + 2511, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 553, 0, 0, 25, 1644, 1632, 0, 0, 1633, 1634, + 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, + 0, 0, 0, 0, 0, 1412, 0, 0, 0, 26, + 1689, 2465, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1710, 2117, 1644, + 0, 1644, 0, 1414, 1414, 0, 1916, 0, 0, 1414, + 0, 0, 1644, 0, 0, 1644, 1107, 0, 0, 0, + 1644, 0, 0, 1644, 0, 2565, 0, 0, 0, 0, + 0, 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 0, 0, 0, 0, 0, + 0, 0, 1891, 0, 27, 28, 29, 0, 0, 0, + 0, 0, 30, 1644, 0, 31, 0, 0, 0, 1413, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1588, 0, 0, 1589, 0, 0, 3278, 1338, 1339, + 0, 0, 1580, 0, 0, 0, 0, 0, 1279, 0, + 0, 0, 0, 0, 32, 0, 0, 0, 0, 0, + 0, 0, 0, 33, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1412, 0, 0, 0, 34, + 0, 0, 0, 0, 0, 0, 35, 0, 0, 0, + 36, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 37, 0, 0, 0, 2670, 0, 0, 1644, 0, 0, + 0, 0, 38, 0, 0, 0, 39, 0, 0, 0, + 0, 0, 0, 1644, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1644, 1644, 1644, 40, 0, 0, 1644, + 0, 0, 0, 1644, 0, 0, 0, 0, 0, 41, + 0, 0, 42, 0, 0, 43, 0, 0, 0, 0, + 44, 0, 0, 0, 0, 0, 0, 0, 0, 1413, + 0, 0, 0, 0, 0, 0, 45, 0, 0, 1363, + 1364, 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, 1603, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1603, - 1603, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3303, 0, 0, 0, 0, 0, - 0, 0, 1591, 0, 2950, -2003, -2003, -2003, 0, 1595, - 1596, 1597, 1598, 1599, 1600, 0, 0, 998, 0, 0, + 46, 0, 0, 0, 0, 1644, 0, 0, 0, 0, + 0, 0, 0, 0, 47, 0, 0, -45, 0, 0, + 1365, 1366, 0, 0, 1367, 1368, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2750, 0, 0, 0, 0, + 0, 2752, 2021, 0, 1644, 0, 0, 0, 0, 0, + 0, 0, 2758, 0, 0, 0, 0, 0, 0, 0, + 1644, 0, 0, 2767, 0, 1644, 2770, 0, 2772, 0, + 0, 0, 0, 0, 0, 0, 2776, 0, 0, 0, + 0, 0, 1916, 0, 2783, 2784, 3318, 0, 0, 0, + 0, 2791, 0, 0, 0, 0, 0, 0, 0, 1369, + 1370, 0, 0, 0, 0, 0, 2800, 0, 0, 1606, + 0, 0, 0, 0, 1607, 0, 2815, 1608, 0, 0, + 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, + 0, 0, 0, 0, 0, 0, 1107, 0, 0, 0, + 0, 0, 1616, 0, 0, 0, 1617, 0, 0, 0, + 0, 0, 1618, 0, 0, 0, 0, 0, 0, 1619, + 0, 0, 0, 1371, 1372, 1373, 1374, 1375, 1376, 1377, + 1378, 0, 0, 1379, 1380, 0, 0, 0, 0, 0, + 1413, 0, 0, 0, 0, 0, 1620, 0, 0, 0, + 0, 0, 0, 2267, 0, 2267, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1603, 0, 0, 0, 0, 1590, 0, - 0, 0, 1340, 1341, 0, 0, 0, 0, 0, 3347, - 0, 0, 0, 0, 0, 1603, 1603, 1603, 0, 0, - 1866, 1866, 1866, 1866, 1866, 1866, 1342, 1343, 0, 1866, - 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 1866, 0, - 0, 0, 0, 1603, 1603, 0, 1566, 0, 0, 1567, - 0, 0, 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1603, 0, 0, 1575, 0, 0, 1603, 0, 0, - 548, 0, 0, 0, 1577, 0, 0, 0, 0, 0, - 0, 1578, 0, 0, 0, 0, 0, 0, 3402, 3402, - 3402, 0, 0, 0, 0, 1591, 0, 0, 1592, 1593, - 1594, 1603, 1595, 1596, 1597, 1598, 1599, 1600, 1579, 0, - 0, 0, 2009, 0, 0, 3402, 0, 0, 0, 0, - 1603, 0, 0, 1603, 1603, 0, 0, 549, 0, 0, - 0, 1866, 1866, 0, 0, 1085, 0, 0, 0, 0, - 0, 1086, 0, 550, 0, 1603, 1371, 1371, 1603, 1098, - 1603, 0, 0, 0, 1603, 0, 3402, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1099, - 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, 0, - 0, 0, 551, 0, 0, 0, 0, 0, 0, 1575, - 0, 0, 552, 0, 1580, 0, 0, 0, 1603, 1577, - 0, 0, 0, 0, 553, 1100, 1578, 0, 0, 554, - 0, 1581, 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1579, 0, 0, 1087, 0, 555, 1583, - 1584, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1585, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1566, 0, 0, 1567, 0, - 0, 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, - 0, 556, 0, 0, 0, 557, 0, 0, 0, 0, - 0, 0, 1586, 1575, 0, 1587, 0, 0, 0, 0, - 1101, 1603, 0, 1577, 0, 0, 0, 0, 0, 1588, - 1578, 1085, 1589, 0, 0, 0, 0, 1086, 0, 0, - 0, 0, 1371, 0, 0, 1098, 0, 0, 0, 1580, - 0, 0, 0, 0, 0, 0, 0, 1579, 0, 0, - 0, 0, 0, 0, 0, 1099, 1581, 0, 0, 1102, - 0, 1582, 0, 0, 558, 0, 0, 0, 0, 1103, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 559, - 0, 1104, 0, 0, 1583, 1584, 0, 0, 1603, 0, - 1603, 0, 0, 0, 0, 0, 1603, 0, 0, 1585, - 0, 1100, 1088, 0, 0, 1603, 0, 0, 1603, 0, - 1603, 0, 560, 1105, 1603, 561, 1590, 1866, 1866, 0, - 0, 1603, 1603, 0, 562, 0, 0, 563, 0, 1603, - 0, 0, 1087, 0, 0, 0, 0, 1586, 1603, 0, - 1587, 0, 0, 1580, 0, 564, 0, 0, 0, 0, - 0, 0, 0, 1603, 1588, 0, 0, 1589, 1106, 565, - 1581, 0, 1090, 0, 1107, 1582, 566, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 567, 0, 0, 0, - 0, 0, 568, 0, 0, 0, 0, 1108, 1583, 1584, - 0, 1371, 1109, 0, 0, 0, 1101, 0, 0, 0, - 0, 0, 0, 1585, 0, 0, 1110, 0, 0, 569, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1591, 0, 0, 1592, 1593, 1594, 0, - 1595, 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, - 2009, 1586, 0, 0, 1587, 1102, 0, 0, 0, 0, - 0, 1590, 0, 0, 0, 1103, 0, 0, 1588, 0, - 0, 1589, 0, 1092, 0, 0, 0, 1104, 0, 0, + 0, 0, 0, 1644, 0, 0, 0, 0, 0, 0, + 0, 1916, 1916, 0, 1414, 1414, 1414, 1414, 1414, 1414, + 0, 0, 1414, 1414, 1414, 1414, 1414, 1414, 1414, 1414, + 1414, 1414, 1916, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1621, 0, 0, 0, 1381, 1382, 0, 0, + 3017, 3018, 0, 0, 0, 0, 0, 0, 0, 1622, + 0, 0, 0, 0, 1623, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1088, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1105, + 0, 0, 0, 0, 0, 3042, 0, 1624, 1625, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 3046, + 0, 0, 1626, 0, 3048, 3049, 0, 0, 0, 3050, + 0, 0, 0, 0, 3053, 0, 0, 3056, 3057, 0, + 0, 0, 2267, 1279, 0, 0, 3065, 0, 0, 0, + 0, 0, 0, 1644, 0, 0, 1644, 1107, 0, 0, + 1627, 1383, 1384, 1628, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1629, 0, 0, + 1630, 0, 0, 0, 0, 1385, 1386, 1607, 0, 0, + 1608, 0, 0, 0, 1609, 1610, 0, 0, 1613, 1614, + 1615, 3106, 1644, 0, 0, 0, 1644, 1644, 1644, 1644, + 1644, 1644, 1644, 1644, 0, 1616, 0, 0, 0, 1414, + 1414, 0, 1644, 1644, 0, 1618, 0, 0, 3125, 0, + 0, 0, 1619, 0, 1644, 0, 0, 1644, 0, 0, + 0, 0, 0, 0, 0, 1644, 1644, 1644, 1644, 1644, + 1644, 1644, 1644, 1644, 1644, 0, 0, 0, 0, 1620, + 1607, 0, 0, 1608, 0, 0, 0, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 1631, 0, 0, + 1644, 0, 0, 0, 0, 0, 0, 0, 1616, 0, + 0, 0, 2762, 0, 0, 0, 0, 0, 1618, 0, + 0, 0, 0, 0, 0, 1619, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 3158, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1620, 1414, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1866, 1371, 0, 0, + 0, 0, 0, 0, 0, 1621, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1588, 0, 0, 0, 0, + 0, 0, 1622, 0, 0, 0, 0, 1623, 0, 0, + 0, 0, 0, 0, 1632, 1279, 0, 1633, 1634, 1635, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 2477, + 1624, 1625, 1644, 0, 0, 0, 0, 0, 0, 0, + 3220, 3221, 1644, 1644, 3222, 1626, 1580, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1621, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 2193, 0, 0, 0, 1090, 0, - 1107, 1603, 1603, 0, 0, 1590, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 0, 0, 1108, 0, 2303, 1603, 0, 2194, 0, - 1603, 0, 1603, 1603, 1603, 0, 0, 1603, 0, 0, - 1603, 1603, 1110, 0, 0, 0, 0, 0, 0, 1603, + 0, 3240, 0, 0, 0, 1622, 0, 0, 0, 0, + 1623, 0, 0, 1627, 0, 0, 1628, 0, 0, 0, + 0, 0, 0, 0, 0, 3252, 0, 0, 0, 0, + 1629, 0, 0, 1624, 1625, 0, 0, 1644, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1626, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1644, + 1644, 1644, 0, 0, 1916, 1916, 1916, 1916, 1916, 1916, + 0, 0, 0, 1916, 1916, 1916, 1916, 1916, 1916, 1916, + 1916, 1916, 1916, 0, 0, 0, 1627, 1644, 1644, 1628, + 0, 0, 0, 0, 0, 3297, 0, 0, 0, 0, + 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 3017, 1644, 1607, 0, 3315, 1608, + 1644, 0, 1107, 1609, 1610, 1611, 1612, 1613, 1614, 1615, + 1631, 0, 0, 0, 3325, 0, 0, 0, 0, 2267, + 0, 2267, 0, 0, 1616, 0, 0, 0, 717, 1107, + 0, 0, 0, 1644, 1618, 0, 0, 0, 0, 0, + 0, 1619, 0, 0, 0, 0, 0, 3350, 0, 0, + 0, 0, 1644, 0, 0, 1644, 1644, 0, 0, 0, + 0, 0, 0, 1916, 1916, 0, 0, 0, 1620, 0, + 0, 0, 0, 0, 0, 0, 0, 1644, 1414, 1414, + 1644, 0, 1644, 1631, 0, 718, 1644, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 3376, 0, 0, 0, + 0, 719, 0, 0, 0, 0, 3017, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1632, 0, 1107, + 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1644, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 720, 3420, 0, 0, 0, 0, 0, 0, 0, 0, + 721, 0, 0, 0, 1621, 0, 0, 0, 0, 0, + 0, 0, 722, 0, 0, 0, 0, 723, 0, 0, + 0, 1622, 0, 0, 0, 0, 1623, 0, 0, 0, + 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 0, 0, 0, 724, 2058, 0, 1624, + 1625, 0, 0, 0, 0, 0, 0, 0, 1607, 0, + 0, 1608, 0, 0, 1626, 1609, 1610, 1611, 1612, 1613, + 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 3475, 3475, 3475, 0, 1644, 1616, 0, 0, 725, + 0, 0, 0, 726, 0, 0, 1618, 0, 0, 0, + 0, 0, 1627, 1619, 0, 1628, 0, 0, 3475, 1414, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1629, + 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, + 1620, 0, 1607, 0, 0, 1608, 0, 0, 0, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 3475, + 0, 0, 0, 0, 0, 0, 0, 539, 0, 0, + 1616, 0, 0, 727, 0, 1644, 0, 1644, 0, 0, + 1618, 0, 0, 1644, 0, 0, 0, 1619, 728, 0, + 0, 0, 1644, 0, 0, 1644, 0, 1644, 0, 0, + 0, 1644, 0, 0, 1916, 1916, 0, 0, 1644, 1644, + 0, 0, 0, 0, 1620, 0, 1644, 0, 0, 0, + 0, 729, 0, 0, 730, 1644, 0, 0, 0, 1631, + 0, 0, 0, 0, 1607, 731, 1621, 1608, 732, 0, + 1644, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, + 0, 0, 0, 1622, 0, 0, 733, 0, 1623, 0, + 0, 0, 1616, 0, 0, 0, 0, 0, 0, 0, + 734, 0, 1618, 0, 0, 0, 735, 736, 1414, 1619, + 0, 1624, 1625, 0, 0, 0, 0, 737, 0, 0, + 0, 0, 0, 738, 0, 0, 1626, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1620, 0, 0, 0, + 1621, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 739, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 0, 0, 1623, 0, 1627, 0, 1632, 1628, 0, 1633, + 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, + 0, 1629, 0, 2058, 1630, 1624, 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1626, 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, - 1603, 0, 0, 0, 0, 0, 1866, 0, 0, 1092, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1603, - 0, 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, - 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, 2404, + 0, 0, 1621, 0, 0, 0, 0, 0, 1627, 0, + 0, 1628, 0, 0, 0, 0, 0, 1916, 1414, 1622, + 0, 0, 0, 0, 1623, 1629, 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1603, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1644, 1644, 0, 0, 0, 1624, 1625, 0, + 0, 1631, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1626, 0, 0, 0, 0, 1644, 0, 0, + 0, 1644, 0, 1644, 1644, 1644, 0, 0, 1644, 0, + 0, 1644, 1644, 0, 0, 0, 0, 0, 0, 0, + 1644, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1627, 0, 0, 1628, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1629, 0, 0, + 1630, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 1644, 0, 0, 0, 0, 0, 1916, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1866, 0, 0, + 1644, 0, 0, 0, 0, 0, 0, 0, 1632, 0, + 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, + 1641, 0, 0, 0, 0, 2363, 0, 0, 0, 0, + 0, 0, 0, 1644, 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, 1916, 0, 0, 0, 0, 0, 1631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1603, 1603, 1603, 0, 0, 0, 0, 0, 0, + 0, 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 1644, 1644, 1644, 0, 2462, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1603, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1603, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1644, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1644, 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, 1603, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1603, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1603, 0, 0, 0, 0, 0, + 0, 0, 1644, 0, 1632, 0, 0, 1633, 1634, 1635, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 0, + 1644, 2486, 0, 0, 0, 0, 0, 0, 0, 0, + 1644, 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, 1603, + 0, 0, 0, 0, 0, 1644, 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, 1603, 0, 110, 933, 642, 934, - 935, 936, 937, 938, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 939, 0, 0, 122, 123, 124, 1603, 125, - 126, 127, 128, 129, 130, 131, 132, 940, 134, 941, - 942, 0, 137, 138, 139, 140, 141, 142, 943, 612, - 143, 144, 145, 146, 944, 945, 149, 0, 150, 151, - 152, 153, 613, 0, 614, 0, 946, 157, 158, 159, - 160, 161, 162, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 1603, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 947, 184, 185, 948, 187, - 949, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 950, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 951, 216, 217, 218, 219, - 220, 615, 952, 222, 0, 223, 224, 953, 226, 0, - 227, 0, 228, 229, 0, 230, 231, 232, 233, 234, - 235, 0, 236, 0, 954, 955, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 261, 262, 263, 264, 265, 266, 267, 956, 957, - 0, 958, 0, 271, 272, 273, 274, 275, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 283, 284, 285, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 959, + 0, 1644, 0, 116, 1042, 815, 1043, 1044, 1045, 1046, + 1047, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 117, 118, 119, 120, 121, 122, 123, 124, + 0, 125, 126, 127, 0, 0, 0, 0, 0, 1048, + 0, 0, 128, 129, 130, 1644, 131, 132, 133, 134, + 135, 136, 137, 138, 1049, 140, 1050, 1051, 0, 143, + 144, 145, 146, 147, 148, 1052, 785, 149, 150, 151, + 152, 1053, 1054, 155, 0, 156, 157, 158, 159, 786, + 0, 787, 0, 1055, 163, 164, 165, 166, 167, 168, + 169, 170, 171, 0, 172, 173, 174, 175, 176, 177, + 1644, 178, 179, 180, 181, 182, 183, 184, 185, 186, + 187, 188, 1056, 190, 191, 1057, 193, 1058, 194, 0, + 195, 196, 197, 198, 199, 200, 0, 0, 201, 202, + 203, 204, 0, 0, 205, 206, 1059, 208, 209, 0, + 210, 211, 212, 0, 213, 214, 215, 0, 216, 217, + 218, 219, 1060, 221, 222, 223, 224, 225, 788, 1061, + 227, 0, 228, 229, 1062, 231, 0, 232, 0, 233, + 234, 0, 235, 236, 237, 238, 239, 240, 0, 241, + 0, 1063, 1064, 244, 245, 0, 246, 247, 248, 249, + 250, 251, 252, 253, 254, 255, 256, 257, 0, 258, + 259, 260, 261, 262, 263, 264, 0, 265, 266, 267, + 268, 269, 270, 271, 272, 1065, 1066, 0, 1067, 0, + 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, + 286, 0, 0, 287, 288, 289, 290, 0, 291, 292, + 293, 294, 295, 296, 297, 298, 1068, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, 314, 315, 316, 317, 318, 319, 1069, 321, 1070, + 323, 324, 325, 326, 1071, 327, 328, 329, 330, 1072, + 790, 332, 1073, 334, 335, 336, 0, 337, 338, 0, + 0, 1074, 340, 341, 0, 0, 342, 343, 344, 345, + 346, 347, 792, 349, 350, 351, 352, 353, 354, 355, + 356, 357, 358, 359, 0, 0, 0, 0, 360, 361, + 793, 363, 364, 365, 366, 367, 368, 369, 0, 370, + 371, 372, 373, 374, 375, 0, 376, 377, 378, 1075, + 380, 381, 382, 383, 0, 384, 385, 386, 387, 388, + 389, 390, 391, 392, 393, 394, 395, 396, 0, 397, + 398, 399, 400, 401, 402, 1076, 404, 405, 406, 407, + 408, 409, 410, 411, 412, 413, 414, 415, 416, 0, + 0, 417, 418, 419, 420, 421, 422, 423, 424, 425, + 0, 426, 427, 428, 1077, 430, 0, 431, 432, 433, + 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, + 444, 795, 0, 0, 446, 447, 0, 448, 449, 450, + 451, 452, 453, 454, 0, 455, 1078, 1079, 0, 0, + 458, 459, 796, 461, 797, 1080, 463, 464, 798, 466, + 467, 468, 469, 470, 0, 0, 471, 472, 473, 0, + 474, 475, 476, 477, 0, 478, 479, 480, 481, 482, + 483, 1081, 0, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, + 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, + 508, 509, 510, 511, 512, 513, 514, 515, 1082, 0, + 0, 0, 0, 0, 0, 1083, 1084, 1085, 0, 0, + 0, 0, 1086, 0, 1087, 0, 0, 0, 0, 1088, + 1089, 1090, 1091, 0, 2617, 116, 1042, 815, 1043, 1044, + 1045, 1046, 1047, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 1048, 0, 0, 128, 129, 130, 0, 131, 132, + 133, 134, 135, 136, 137, 138, 1049, 140, 1050, 1051, + 0, 143, 144, 145, 146, 147, 148, 1052, 785, 149, + 150, 151, 152, 1053, 1054, 155, 0, 156, 157, 158, + 159, 786, 0, 787, 0, 1055, 163, 164, 165, 166, + 167, 168, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 178, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 1056, 190, 191, 1057, 193, 1058, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 1059, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 1060, 221, 222, 223, 224, 225, + 788, 1061, 227, 0, 228, 229, 1062, 231, 0, 232, + 0, 233, 234, 0, 235, 236, 237, 238, 239, 240, + 0, 241, 0, 1063, 1064, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 266, 267, 268, 269, 270, 271, 272, 1065, 1066, 0, + 1067, 0, 276, 277, 278, 279, 280, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 288, 289, 290, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 1068, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 1069, + 321, 1070, 323, 324, 325, 326, 1071, 327, 328, 329, + 330, 1072, 790, 332, 1073, 334, 335, 336, 0, 337, + 338, 0, 0, 1074, 340, 341, 0, 0, 342, 343, + 344, 345, 346, 347, 792, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 793, 363, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 1075, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 399, 400, 401, 402, 1076, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 426, 427, 428, 1077, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, 444, 795, 0, 0, 446, 447, 0, 448, + 449, 450, 451, 452, 453, 454, 0, 455, 1078, 1079, + 0, 0, 458, 459, 796, 461, 797, 1080, 463, 464, + 798, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 1081, 0, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 1082, 0, 0, 0, 0, 0, 0, 1083, 1084, 1085, + 0, 0, 0, 0, 1086, 0, 1087, 0, 0, 0, + 0, 1088, 1089, 1090, 1091, 0, 3223, 116, 1042, 815, + 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 1048, 0, 0, 128, 129, 130, 0, + 131, 132, 133, 134, 135, 136, 137, 138, 1049, 140, + 1050, 1051, 0, 143, 144, 145, 146, 147, 148, 1052, + 785, 149, 150, 151, 152, 1053, 1054, 155, 0, 156, + 157, 158, 159, 786, 0, 787, 0, 1055, 163, 164, + 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 178, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 1056, 190, 191, 1057, + 193, 1058, 194, 0, 195, 196, 197, 198, 199, 200, + 14, 15, 201, 202, 203, 204, 0, 0, 205, 206, + 1059, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 1060, 221, 222, 223, + 224, 225, 788, 1061, 227, 0, 228, 229, 1062, 231, + 0, 232, 0, 233, 234, 23, 235, 236, 237, 238, + 239, 240, 0, 241, 0, 1063, 1064, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 266, 267, 268, 269, 270, 271, 272, 1065, + 1066, 0, 1067, 0, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 288, 289, + 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 1068, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, 327, + 328, 329, 330, 1072, 790, 332, 1073, 334, 335, 336, + 0, 337, 338, 0, 0, 1074, 340, 341, 0, 0, + 342, 343, 344, 345, 346, 347, 792, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 27, 28, + 29, 0, 360, 361, 793, 363, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 1075, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 399, 400, 401, 402, 1076, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 34, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 36, 426, 427, 428, 1077, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, 444, 795, 38, 0, 446, 447, + 39, 448, 449, 450, 451, 452, 453, 454, 0, 455, + 1078, 1079, 0, 0, 458, 459, 796, 461, 797, 1080, + 463, 464, 798, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 41, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 799, 1081, 0, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 45, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 1082, 0, 46, 0, 0, 0, 0, 1083, + 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, 3197, + 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, 815, + 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 1048, 0, 0, 128, 129, 130, 0, + 131, 132, 133, 134, 135, 136, 137, 138, 1049, 140, + 1050, 1051, 0, 143, 144, 145, 146, 147, 148, 1052, + 785, 149, 150, 151, 152, 1053, 1054, 155, 0, 156, + 157, 158, 159, 786, 0, 787, 0, 1055, 163, 164, + 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 178, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 1056, 190, 191, 1057, + 193, 1058, 194, 0, 195, 196, 197, 198, 199, 200, + 14, 15, 201, 202, 203, 204, 0, 0, 205, 206, + 1059, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 1060, 221, 222, 223, + 224, 225, 788, 1061, 227, 0, 228, 229, 1062, 231, + 0, 232, 0, 233, 234, 23, 235, 236, 237, 238, + 239, 240, 0, 241, 0, 1063, 1064, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 266, 267, 268, 269, 270, 271, 272, 1065, + 1066, 0, 1067, 0, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 288, 289, + 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 1068, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, 327, + 328, 329, 330, 1072, 790, 332, 1073, 334, 335, 336, + 0, 337, 338, 0, 0, 1074, 340, 341, 0, 0, + 342, 343, 344, 345, 346, 347, 792, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 27, 28, + 29, 0, 360, 361, 793, 363, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 1075, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 399, 400, 401, 402, 1076, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 34, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 36, 426, 427, 428, 1077, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, 444, 795, 38, 0, 446, 447, + 39, 448, 449, 450, 451, 452, 453, 454, 0, 455, + 1078, 1079, 0, 0, 458, 459, 796, 461, 797, 1080, + 463, 464, 798, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 41, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 799, 1081, 0, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 45, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 1082, 0, 46, 0, 0, 0, 0, 1083, + 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, 0, + 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, 815, + 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 1048, 0, 0, 128, 129, 130, 0, + 131, 132, 133, 134, 135, 136, 137, 138, 1049, 140, + 1050, 1051, 0, 143, 144, 145, 146, 147, 148, 1052, + 785, 149, 150, 151, 152, 1053, 1054, 155, 0, 156, + 157, 158, 159, 786, 0, 787, 0, 1055, 163, 164, + 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 178, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 1056, 190, 191, 1057, + 193, 1058, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 1059, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 1060, 221, 222, 223, + 224, 225, 788, 1061, 227, 0, 228, 229, 1062, 231, + 0, 232, 0, 233, 234, 23, 235, 236, 237, 238, + 239, 240, 0, 241, 0, 1063, 1064, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 266, 267, 268, 269, 270, 271, 272, 1065, + 1066, 0, 1067, 0, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 288, 289, + 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 1068, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, 327, + 328, 329, 330, 1072, 790, 332, 1073, 334, 335, 336, + 0, 337, 338, 0, 0, 1074, 340, 341, 0, 0, + 342, 343, 344, 345, 346, 347, 792, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 27, 28, + 29, 0, 360, 361, 793, 363, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 1075, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 399, 400, 401, 402, 1076, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 34, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 426, 427, 428, 1077, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, 444, 795, 0, 0, 446, 447, + 39, 448, 449, 450, 451, 452, 453, 454, 0, 455, + 1078, 1079, 0, 0, 458, 459, 796, 461, 797, 1080, + 463, 464, 798, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 41, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 799, 1081, 0, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 45, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 1082, 0, 46, 0, 0, 0, 0, 1083, + 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, 0, + 0, 0, 0, 1088, 1089, 1090, 1091, 1240, 1042, 815, + 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 1241, 125, 126, 127, 0, 0, + 0, 1242, 0, 1048, 0, 0, 1243, 129, 130, 0, + 131, 132, 133, 1244, 135, 136, 137, 138, 1049, 1245, + 1050, 1051, 0, 143, 144, 145, 146, 147, 148, 1052, + 785, 149, 150, 151, 152, 1053, 1054, 155, 0, 156, + 157, 158, 159, 786, 0, 1246, 0, 1247, 163, 164, + 165, 166, 167, 1248, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 1249, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 1056, 190, 191, 1057, + 193, 1058, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 1250, 0, 205, 206, + 1059, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 1060, 221, 222, 223, + 224, 225, 788, 1061, 227, 0, 228, 229, 1062, 231, + 0, 232, 0, 233, 1251, 0, 1252, 236, 237, 1253, + 1254, 240, 0, 241, 0, 1063, 1064, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 1255, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 1256, 267, 268, 269, 270, 271, 272, 1065, + 1066, 0, 1067, 0, 276, 1257, 1258, 279, 1259, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 1260, 289, + 1261, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 1262, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 1069, 1263, 1070, 323, 324, 325, 326, 1071, 327, + 328, 1264, 330, 1072, 790, 332, 1073, 334, 335, 336, + 0, 337, 338, 0, 0, 1074, 340, 341, 0, 0, + 342, 343, 344, 1265, 346, 1266, 792, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 793, 1267, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 1075, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 1268, 400, 401, 402, 1076, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 1269, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 1270, 427, 428, 1077, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 1271, 444, 795, 0, 0, 446, 447, + 0, 448, 1272, 450, 451, 452, 453, 454, 0, 455, + 1078, 1079, 0, 0, 458, 459, 796, 461, 797, 1080, + 463, 464, 1273, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 1081, 1274, 485, 1275, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 1082, 0, 0, 0, 0, 0, 0, 1083, + 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, 1276, + 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, 815, + 1043, 1044, 0, 1046, 1047, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 1048, 0, 0, 128, 129, 130, 0, + 131, 132, 133, 134, 135, 136, 137, 138, 1049, 140, + 1050, 1051, 0, 143, 144, 145, 146, 147, 148, 1052, + 785, 149, 150, 151, 152, 1053, 1054, 155, 0, 156, + 157, 158, 159, 786, 0, 787, 0, 162, 163, 164, + 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 178, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 1056, 190, 191, 1057, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 14, 15, 201, 202, 203, 204, 0, 0, 205, 206, + 1059, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 1060, 221, 222, 223, + 224, 225, 788, 1061, 227, 0, 228, 229, 1062, 231, + 0, 232, 0, 233, 234, 23, 235, 236, 237, 238, + 239, 240, 0, 241, 0, 1063, 1064, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 253, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 266, 267, 268, 269, 270, 271, 272, 1065, + 1066, 0, 1067, 0, 276, 277, 278, 279, 280, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 288, 289, + 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 1068, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 1069, 321, 1070, 323, 324, 325, 326, 0, 327, + 328, 329, 330, 1072, 790, 332, 1073, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 345, 346, 347, 792, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 27, 28, + 29, 0, 360, 361, 793, 363, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 1075, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 399, 400, 401, 402, 1076, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 34, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 36, 426, 427, 428, 1077, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, 444, 795, 38, 0, 446, 447, + 39, 448, 449, 450, 451, 452, 453, 454, 0, 455, + 1078, 1079, 0, 0, 458, 459, 796, 461, 797, 1080, + 463, 464, 798, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 41, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 799, 1081, 0, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 45, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 0, 0, 46, 0, 0, 0, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 1086, 0, 1087, 0, + 0, 0, 0, 1088, 1089, 1090, 1091, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 1421, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 1422, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 1423, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 1424, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 1240, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 1242, 0, 1048, 0, 0, 1243, 129, 130, + 0, 131, 132, 133, 1244, 135, 136, 137, 138, 1049, + 1245, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 1246, 0, 1247, 163, + 164, 165, 166, 167, 1248, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 1249, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 1251, 0, 1252, 236, 237, + 1253, 1254, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 1255, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 1256, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 1257, 1258, 279, 1259, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 1260, + 289, 1261, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1262, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 1263, 1070, 323, 324, 325, 326, 1071, + 327, 328, 1264, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 1265, 346, 1266, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 1267, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 1268, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 1269, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 1270, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 1271, 444, 795, 0, 0, 446, + 447, 0, 448, 1272, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 1273, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 2272, 485, 1275, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 1240, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 1242, 0, 1048, 0, 0, 1243, 129, 130, + 0, 131, 132, 133, 1244, 135, 136, 137, 138, 1049, + 1245, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 1246, 0, 1247, 163, + 164, 165, 166, 167, 1248, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 1249, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 1251, 0, 1252, 236, 237, + 1253, 1254, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 1255, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 1256, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 1257, 1258, 279, 1259, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 1260, + 289, 1261, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1262, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 1263, 1070, 323, 324, 325, 326, 1071, + 327, 328, 1264, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 1265, 346, 1266, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 1267, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 1268, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 1269, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 1270, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 1271, 444, 795, 0, 0, 446, + 447, 0, 448, 1272, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 1273, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 1275, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 2326, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, -1106, 125, 126, 127, 0, + 0, 0, 0, -1106, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, -1106, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 1240, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 1242, 0, 1048, 0, 0, 1243, 129, 130, + 0, 131, 132, 133, 1244, 135, 136, 137, 138, 1049, + 1245, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 1246, 0, 1247, 163, + 164, 165, 166, 167, 1248, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 1249, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 1251, 0, 1252, 236, 237, + 1253, 1254, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 1255, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 1256, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 1257, 1258, 279, 1259, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 1260, + 289, 1261, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1262, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 1263, 1070, 323, 324, 325, 326, 1071, + 327, 328, 1264, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 1265, 346, 1266, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 1267, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 1268, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 1269, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 1270, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 1271, 444, 795, 0, 0, 446, + 447, 0, 448, 1272, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 1273, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 1275, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 3062, 0, 0, 0, 1088, 1089, 1090, 1091, 1240, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 1242, 0, 1048, 0, 0, 1243, 129, 130, + 0, 131, 132, 133, 1244, 135, 136, 137, 138, 1049, + 1245, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 1246, 0, 1247, 163, + 164, 165, 166, 167, 1248, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 1249, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 1251, 0, 1252, 236, 237, + 1253, 1254, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 1255, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 1256, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 1257, 1258, 279, 1259, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 1260, + 289, 1261, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1262, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 1263, 1070, 323, 324, 325, 326, 1071, + 327, 328, 1264, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 1265, 346, 1266, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 1267, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 1268, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 1269, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 1270, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 1271, 444, 795, 0, 0, 446, + 447, 0, 448, 1272, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 1273, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 1275, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 1686, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1701, + 815, 1043, 1044, 1045, 1702, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 1703, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 1422, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 2046, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 2669, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 2790, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 3005, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 3006, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 3007, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 3008, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 3009, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 3157, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 3296, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1706, 1707, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 2183, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 2266, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 2476, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 3060, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 3006, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 3007, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 3008, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 3009, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 3472, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 3473, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 3474, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 3473, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 1071, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 3474, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 1082, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 162, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 0, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 0, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 0, 0, 0, 0, 0, 0, 0, + 1408, 1409, 0, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, -2041, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 1055, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 3473, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 1058, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, -2041, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, -2041, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 0, 0, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, -2041, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 0, + 327, 328, 0, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, -2041, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 3474, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, -2041, 0, 0, 0, 0, 0, 0, + 1083, 1084, 1085, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 0, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 162, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 0, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 0, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 339, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 2169, 2170, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 0, 0, 0, 0, 0, 0, 0, + 2171, 2172, 0, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 1045, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 162, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 0, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 0, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 0, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 1074, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 0, 0, 0, 0, 0, 0, 0, + 1408, 1409, 0, 0, 0, 0, 0, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 116, 1042, + 815, 1043, 1044, 0, 1046, 1047, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 117, 118, 119, + 120, 121, 122, 123, 124, 0, 125, 126, 127, 0, + 0, 0, 0, 0, 1048, 0, 0, 128, 129, 130, + 0, 131, 132, 133, 134, 135, 136, 137, 138, 1049, + 140, 1050, 1051, 0, 143, 144, 145, 146, 147, 148, + 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, 0, + 156, 157, 158, 159, 786, 0, 787, 0, 162, 163, + 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, + 173, 174, 175, 176, 177, 0, 178, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1056, 190, 191, + 1057, 193, 0, 194, 0, 195, 196, 197, 198, 199, + 200, 0, 0, 201, 202, 203, 204, 0, 0, 205, + 206, 1059, 208, 209, 0, 210, 211, 212, 0, 213, + 214, 215, 0, 216, 217, 218, 219, 1060, 221, 222, + 223, 224, 225, 788, 1061, 227, 0, 228, 229, 1062, + 231, 0, 232, 0, 233, 234, 0, 235, 236, 237, + 238, 239, 240, 0, 241, 0, 1063, 1064, 244, 245, + 0, 246, 247, 248, 249, 250, 251, 252, 253, 254, + 255, 256, 257, 0, 258, 259, 260, 261, 262, 263, + 264, 0, 265, 266, 267, 268, 269, 270, 271, 272, + 1065, 1066, 0, 1067, 0, 276, 277, 278, 279, 280, + 281, 282, 283, 284, 285, 286, 0, 0, 287, 288, + 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, + 298, 1068, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1069, 321, 1070, 323, 324, 325, 326, 0, + 327, 328, 329, 330, 1072, 790, 332, 1073, 334, 335, + 336, 0, 337, 338, 0, 0, 339, 340, 341, 0, + 0, 342, 343, 344, 345, 346, 347, 792, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 0, + 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, + 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, + 0, 376, 377, 378, 1075, 380, 381, 382, 383, 0, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 0, 397, 398, 399, 400, 401, 402, + 1076, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 0, 0, 417, 418, 419, 420, + 421, 422, 423, 424, 425, 0, 426, 427, 428, 1077, + 430, 0, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 443, 444, 795, 0, 0, 446, + 447, 0, 448, 449, 450, 451, 452, 453, 454, 0, + 455, 1078, 1079, 0, 0, 458, 459, 796, 461, 797, + 1080, 463, 464, 798, 466, 467, 468, 469, 470, 0, + 0, 471, 472, 473, 0, 474, 475, 476, 477, 0, + 478, 479, 480, 481, 482, 483, 1081, 0, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 0, 0, 494, + 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, + 503, 504, 505, 506, 507, 508, 509, 510, 511, 512, + 513, 514, 515, 0, 0, 0, 0, 0, 0, 116, + 1042, 815, 1043, 1044, 1045, 1046, 1047, 1086, 0, 1087, + 0, 0, 0, 0, 1088, 1089, 1090, 1091, 117, 118, + 119, 120, 121, 122, 123, 124, 0, 125, 126, 127, + 0, 0, 0, 0, 0, 1048, 0, 0, 128, 129, + 130, 0, 131, 132, 133, 134, 135, 136, 137, 0, + 1049, 140, 1050, 1051, 0, 143, 144, 145, 146, 147, + 148, 1052, 785, 149, 150, 151, 152, 1053, 1054, 155, + 0, 156, 157, 158, 159, 786, 0, 787, 0, 1055, + 163, 164, 165, 166, 167, 168, 169, 170, 171, 0, + 172, 173, 174, 175, 176, 177, 0, 178, 179, 180, + 181, 182, 183, 184, 185, 186, 187, 188, 1056, 190, + 191, 1057, 193, 1058, 194, 0, 195, 196, 197, 198, + 199, 200, 0, 0, 201, 202, 203, 204, 0, 0, + 205, 206, 1059, 208, 209, 0, 210, 211, 212, 0, + 213, 0, 215, 0, 216, 217, 218, 219, 1060, 221, + 222, 223, 224, 225, 788, 1061, 227, 0, 228, 229, + 1062, 231, 0, 232, 0, 233, 234, 0, 235, 236, + 237, 238, 0, 240, 0, 241, 0, 1063, 1064, 244, + 245, 0, 246, 247, 248, 249, 250, 251, 252, 0, + 254, 255, 256, 257, 0, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 266, 267, 268, 269, 270, 271, + 272, 1065, 1066, 0, 1067, 0, 276, 0, 0, 279, + 280, 281, 282, 283, 284, 285, 286, 0, 0, 287, + 288, 289, 0, 0, 291, 292, 293, 294, 295, 296, + 297, 298, 1068, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 1069, 321, 1070, 323, 324, 325, 326, + 0, 327, 328, 0, 330, 1072, 790, 332, 1073, 334, + 335, 336, 0, 337, 338, 0, 0, 1074, 340, 341, + 0, 0, 342, 343, 344, 345, 346, 347, 792, 349, + 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, + 0, 0, 0, 0, 360, 361, 793, 363, 364, 365, + 366, 367, 368, 369, 0, 370, 371, 372, 373, 374, + 375, 0, 376, 377, 378, 1075, 380, 381, 382, 383, + 0, 384, 385, 386, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 0, 397, 398, 399, 400, 401, + 402, 1076, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 415, 416, 0, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 0, 0, 427, 428, + 1077, 430, 0, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 443, 444, 795, 0, 0, + 446, 447, 0, 448, 449, 450, 451, 452, 453, 454, + 0, 455, 1078, 1079, 0, 0, 458, 459, 796, 461, + 797, 1080, 463, 464, 798, 466, 467, 468, 469, 470, + 0, 0, 471, 472, 473, 0, 474, 475, 476, 477, + 0, 478, 479, 480, 481, 482, 483, 1081, 0, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 0, 0, + 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, + 502, 503, 504, 505, 506, 507, 508, 509, 510, 511, + 512, 513, 514, 515, 0, 0, 0, 0, 0, 0, + 0, 1083, 1084, 1085, 0, 964, 1339, 815, 1086, 0, + 1087, 1046, 0, 0, 0, 1088, 1089, 1090, 1091, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 560, + 0, 0, 0, 0, 565, 129, 130, 0, 131, 132, + 133, 567, 135, 136, 137, 568, 569, 570, 571, 572, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 576, 577, 155, 0, 156, 157, 158, + 159, 579, 0, 581, 0, 583, 163, 164, 165, 166, + 167, 584, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 587, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 589, 190, 191, 590, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 600, 221, 222, 223, 224, 225, + 601, 1340, 227, 0, 228, 229, 604, 231, 0, 232, + 0, 233, 607, 0, 609, 236, 237, 610, 611, 240, + 0, 241, 0, 614, 615, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 617, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 620, 621, 268, 269, 270, 271, 272, 622, 623, 0, + 625, 0, 276, 627, 628, 279, 629, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 632, 289, 633, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 635, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 636, + 637, 638, 323, 324, 325, 639, 0, 327, 328, 641, + 330, 0, 643, 332, 644, 334, 335, 336, 0, 337, + 338, 1341, 0, 339, 340, 341, 0, 0, 342, 343, + 650, 651, 346, 652, 653, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 658, 659, 364, 365, 660, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 663, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 666, 400, 401, 402, 667, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 669, 417, 418, 419, 420, 421, 422, 670, + 424, 425, 0, 672, 427, 428, 673, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 675, 444, 676, 0, 0, 446, 447, 0, 448, + 680, 450, 451, 452, 453, 454, 0, 455, 682, 683, + 0, 0, 458, 459, 686, 461, 687, 1342, 463, 464, + 689, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 694, 695, 0, 485, 697, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 512, 513, 514, 515, + 0, 0, 0, 0, 0, 0, 0, 1343, 1344, 2344, + 0, 116, 1042, 815, 1043, 1044, 2345, 1046, 0, 0, + 0, 0, 0, 0, 1091, 0, 0, 0, 0, 0, + 117, 118, 119, 120, 121, 122, 123, 124, 0, 125, + 126, 127, 0, 0, 0, 0, 0, 1048, 0, 0, + 128, 129, 130, 0, 131, 132, 133, 134, 135, 136, + 137, 138, 1049, 140, 1050, 1051, 0, 143, 144, 145, + 146, 147, 148, 1052, 0, 149, 150, 151, 152, 1053, + 1054, 155, 0, 156, 157, 158, 159, 160, 0, 161, + 0, 162, 163, 164, 165, 166, 167, 168, 169, 170, + 171, 0, 172, 173, 174, 175, 176, 177, 0, 178, + 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, + 1056, 190, 191, 1057, 193, 0, 194, 0, 195, 196, + 197, 198, 199, 200, 0, 0, 201, 202, 203, 204, + 0, 0, 205, 206, 1059, 208, 209, 0, 210, 211, + 212, 0, 213, 214, 215, 0, 216, 217, 218, 219, + 1060, 221, 222, 223, 224, 225, 226, 1061, 227, 0, + 228, 229, 1062, 231, 0, 232, 0, 233, 234, 0, + 235, 236, 237, 238, 239, 240, 0, 241, 3026, 1063, + 1064, 244, 245, 0, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 0, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 266, 267, 268, 269, + 270, 271, 272, 1065, 1066, 0, 1067, 0, 276, 277, + 278, 279, 280, 281, 282, 283, 284, 285, 286, 0, + 0, 287, 288, 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 960, 316, 961, 318, 319, 320, 321, 962, 322, 323, - 324, 325, 963, 617, 327, 964, 329, 330, 331, 0, - 332, 333, 0, 0, 965, 335, 336, 0, 0, 337, - 338, 339, 340, 341, 342, 619, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 620, 357, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 966, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 393, 394, 395, 396, 967, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, 420, 421, 968, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 622, 439, 0, 440, 441, 0, 442, 443, 444, - 445, 446, 447, 448, 0, 449, 969, 970, 0, 0, - 452, 453, 623, 455, 624, 971, 457, 458, 625, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 972, 0, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 973, 0, - 0, 0, 0, 0, 0, 974, 975, 976, 0, 0, - 0, 0, 977, 0, 978, 0, 0, 0, 0, 979, - 980, 981, 982, 0, 2559, 110, 933, 642, 934, 935, - 936, 937, 938, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 939, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 940, 134, 941, 942, - 0, 137, 138, 139, 140, 141, 142, 943, 612, 143, - 144, 145, 146, 944, 945, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 946, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 947, 184, 185, 948, 187, 949, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 951, 216, 217, 218, 219, 220, - 615, 952, 222, 0, 223, 224, 953, 226, 0, 227, - 0, 228, 229, 0, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 954, 955, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 956, 957, 0, - 958, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 283, 284, 285, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 959, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 960, - 316, 961, 318, 319, 320, 321, 962, 322, 323, 324, - 325, 963, 617, 327, 964, 329, 330, 331, 0, 332, - 333, 0, 0, 965, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, + 315, 316, 317, 318, 319, 1069, 321, 1070, 323, 324, + 325, 326, 0, 327, 328, 329, 330, 1072, 331, 332, + 1073, 334, 335, 336, 0, 337, 338, 0, 0, 339, + 340, 341, 0, 0, 342, 343, 344, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + 358, 359, 0, 0, 0, 0, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 966, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 967, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 968, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 969, 970, 0, 0, 452, - 453, 623, 455, 624, 971, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 972, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 973, 0, 0, - 0, 0, 0, 0, 974, 975, 976, 0, 0, 0, - 0, 977, 0, 978, 0, 0, 0, 0, 979, 980, - 981, 982, 0, 3153, 110, 933, 642, 934, 935, 936, - 937, 938, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 939, 0, 0, 122, 123, 124, 0, 125, 126, 127, - 128, 129, 130, 131, 132, 940, 134, 941, 942, 0, - 137, 138, 139, 140, 141, 142, 943, 612, 143, 144, - 145, 146, 944, 945, 149, 0, 150, 151, 152, 153, - 613, 0, 614, 0, 946, 157, 158, 159, 160, 161, - 162, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 172, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 947, 184, 185, 948, 187, 949, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 950, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 951, 216, 217, 218, 219, 220, 615, - 952, 222, 0, 223, 224, 953, 226, 0, 227, 0, - 228, 229, 21, 230, 231, 232, 233, 234, 235, 0, - 236, 0, 954, 955, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 261, - 262, 263, 264, 265, 266, 267, 956, 957, 0, 958, - 0, 271, 272, 273, 274, 275, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 283, 284, 285, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 959, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 960, 316, - 961, 318, 319, 320, 321, 962, 322, 323, 324, 325, - 963, 617, 327, 964, 329, 330, 331, 0, 332, 333, - 0, 0, 965, 335, 336, 0, 0, 337, 338, 339, - 340, 341, 342, 619, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 25, 26, 27, 0, 354, 355, - 620, 357, 358, 359, 360, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 966, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 393, 394, 395, 396, 967, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 32, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, - 420, 421, 968, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 622, - 439, 0, 440, 441, 37, 442, 443, 444, 445, 446, - 447, 448, 0, 449, 969, 970, 0, 0, 452, 453, - 623, 455, 624, 971, 457, 458, 625, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 39, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 626, 972, - 0, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 43, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 973, 0, 44, 0, - 0, 0, 0, 974, 975, 976, 0, 0, 0, 0, - 977, 0, 978, 3127, 0, 0, 0, 979, 980, 981, - 982, 110, 933, 642, 934, 935, 936, 937, 938, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 0, 0, 939, 0, 0, - 122, 123, 124, 0, 125, 126, 127, 128, 129, 130, - 131, 132, 940, 134, 941, 942, 0, 137, 138, 139, - 140, 141, 142, 943, 612, 143, 144, 145, 146, 944, - 945, 149, 0, 150, 151, 152, 153, 613, 0, 614, - 0, 946, 157, 158, 159, 160, 161, 162, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 947, 184, 185, 948, 187, 949, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 950, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 951, 216, 217, 218, 219, 220, 615, 952, 222, 0, - 223, 224, 953, 226, 0, 227, 0, 228, 229, 21, - 230, 231, 232, 233, 234, 235, 0, 236, 0, 954, - 955, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 261, 262, 263, 264, - 265, 266, 267, 956, 957, 0, 958, 0, 271, 272, - 273, 274, 275, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 283, 284, 285, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 959, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 960, 316, 961, 318, 319, - 320, 321, 962, 322, 323, 324, 325, 963, 617, 327, - 964, 329, 330, 331, 0, 332, 333, 0, 0, 965, - 335, 336, 0, 0, 337, 338, 339, 340, 341, 342, - 619, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 25, 26, 27, 0, 354, 355, 620, 357, 358, - 359, 360, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 966, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 393, 394, - 395, 396, 967, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 32, 0, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 419, 420, 421, 968, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 622, 439, 0, 440, - 441, 37, 442, 443, 444, 445, 446, 447, 448, 0, - 449, 969, 970, 0, 0, 452, 453, 623, 455, 624, - 971, 457, 458, 625, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 39, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 626, 972, 0, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 43, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, - 507, 508, 509, 973, 0, 44, 0, 0, 0, 0, - 974, 975, 976, 0, 0, 0, 0, 977, 0, 978, - 0, 0, 0, 0, 979, 980, 981, 982, 1207, 933, - 642, 934, 935, 936, 937, 938, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 1208, 119, 120, 121, 0, - 0, 0, 1209, 0, 939, 0, 0, 1210, 123, 124, - 0, 125, 126, 127, 1211, 129, 130, 131, 132, 940, - 1212, 941, 942, 0, 137, 138, 139, 140, 141, 142, - 943, 612, 143, 144, 145, 146, 944, 945, 149, 0, - 150, 151, 152, 153, 613, 0, 1213, 0, 1214, 157, - 158, 159, 160, 161, 1215, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 1216, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 947, 184, 185, - 948, 187, 949, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 1217, 0, 200, - 201, 950, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 951, 216, 217, - 218, 219, 220, 615, 952, 222, 0, 223, 224, 953, - 226, 0, 227, 0, 228, 1218, 0, 1219, 231, 232, - 1220, 1221, 235, 0, 236, 0, 954, 955, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 1222, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 1223, 262, 263, 264, 265, 266, 267, - 956, 957, 0, 958, 0, 271, 1224, 1225, 274, 1226, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 1227, - 284, 1228, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 1229, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 960, 1230, 961, 318, 319, 320, 321, 962, - 322, 323, 1231, 325, 963, 617, 327, 964, 329, 330, - 331, 0, 332, 333, 0, 0, 965, 335, 336, 0, - 0, 337, 338, 339, 1232, 341, 1233, 619, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 620, 1234, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 966, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 1235, 394, 395, 396, 967, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 1236, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 1237, 420, 421, 968, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 1238, 437, 622, 439, 0, 440, 441, 0, 442, - 1239, 444, 445, 446, 447, 448, 0, 449, 969, 970, - 0, 0, 452, 453, 623, 455, 624, 971, 457, 458, - 1240, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 972, 1241, 479, 1242, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, + 373, 374, 375, 0, 376, 377, 378, 1075, 380, 381, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 0, 397, 398, 399, + 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 0, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 0, 426, + 427, 428, 1077, 430, 0, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, 444, 445, + 0, 0, 446, 447, 0, 448, 449, 450, 451, 452, + 453, 454, 0, 455, 1078, 1079, 0, 0, 458, 459, + 460, 461, 462, 1080, 463, 464, 465, 466, 467, 468, + 469, 470, 0, 0, 471, 472, 473, 0, 474, 475, + 476, 477, 0, 478, 479, 480, 481, 482, 483, 1081, + 0, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 973, 0, 0, 0, 0, 0, 0, 974, 975, 976, - 0, 0, 0, 0, 977, 0, 978, 1243, 0, 0, - 0, 979, 980, 981, 982, 110, 933, 642, 934, 935, - 936, 937, 938, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 939, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 940, 134, 941, 942, - 1378, 137, 138, 139, 140, 141, 142, 943, 612, 143, - 144, 145, 146, 944, 945, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 946, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 947, 184, 185, 948, 187, 949, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 951, 216, 217, 218, 219, 220, - 615, 952, 222, 0, 223, 224, 953, 226, 0, 227, - 0, 228, 229, 1379, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 954, 955, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 956, 957, 0, - 958, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 0, 1380, 282, 283, 284, 285, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 959, 295, + 510, 511, 512, 513, 514, 515, 0, 0, 0, 0, + 116, 1042, 815, 1043, 1044, 0, 1046, 0, 0, 0, + 1086, 0, 2719, 0, 0, 0, 0, 1088, 1089, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 1048, 0, 0, 128, + 129, 130, 0, 131, 132, 133, 134, 135, 136, 137, + 138, 1049, 140, 1050, 1051, 0, 143, 144, 145, 146, + 147, 148, 1052, 0, 149, 150, 151, 152, 1053, 1054, + 155, 0, 156, 157, 158, 159, 160, 0, 161, 0, + 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 178, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 1056, + 190, 191, 1057, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 1059, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 1060, + 221, 222, 223, 224, 225, 226, 1061, 227, 0, 228, + 229, 1062, 231, 0, 232, 0, 233, 234, 0, 235, + 236, 237, 238, 239, 240, 0, 241, 0, 1063, 1064, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 266, 267, 268, 269, 270, + 271, 272, 1065, 1066, 0, 1067, 0, 276, 277, 278, + 279, 280, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 288, 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 960, - 316, 961, 318, 319, 320, 321, 962, 322, 323, 324, - 325, 963, 617, 327, 964, 329, 330, 331, 0, 332, - 333, 0, 0, 965, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 966, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 967, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 968, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 969, 970, 0, 1381, 452, - 453, 623, 455, 624, 971, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 972, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 973, 0, 0, - 0, 0, 0, 0, 974, 975, 976, 0, 0, 0, - 0, 977, 0, 978, 0, 0, 0, 0, 979, 980, - 981, 982, 1207, 933, 642, 934, 935, 936, 937, 938, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 1069, 321, 1070, 323, 324, 325, + 326, 0, 327, 328, 329, 330, 1072, 331, 332, 1073, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 345, 346, 347, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 1075, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 426, 427, + 428, 1077, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 443, 444, 445, 0, + 0, 446, 447, 0, 448, 449, 450, 451, 452, 453, + 454, 0, 455, 1078, 1079, 0, 0, 458, 459, 460, + 461, 462, 1080, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 1081, 0, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 0, 0, 0, 0, 523, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1086, + 0, 2719, 0, 0, 0, 0, 1088, 1089, 117, 118, + 119, 120, 121, 122, 123, 124, 0, 125, 126, 127, + 0, 0, 0, 0, 0, 0, 1000, 0, 0, 129, + 130, 0, 131, 132, 133, 0, 135, 136, 137, 138, + 139, 0, 141, 142, 0, 143, 144, 145, 146, 147, + 148, 0, 0, 149, 150, 151, 152, 153, 154, 155, + 0, 156, 157, 158, 159, 160, 0, 0, 0, 162, + 163, 164, 165, 166, 167, 0, 169, 170, 171, 0, + 172, 173, 174, 175, 176, 177, 0, 0, 179, 180, + 181, 182, 183, 184, 185, 186, 187, 188, 189, 190, + 191, 192, 193, 0, 194, 0, 195, 196, 197, 198, + 199, 200, 0, 0, 201, 202, 203, 204, 0, 0, + 205, 206, 207, 208, 209, 0, 210, 211, 212, 0, + 213, 214, 215, -524, 216, 217, 218, 219, 220, 221, + 222, 223, 224, 225, 226, 0, 227, -524, 228, 229, + 230, 231, -524, 232, 0, 233, 0, 0, 0, 236, + 237, 524, 0, 240, 0, 241, 0, 242, 243, 244, + 245, 0, 246, 247, 248, 249, 250, 251, 252, 0, + 254, 255, 256, 257, 0, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 0, 267, 268, 269, 270, 271, + 272, 273, 274, -524, 275, 0, 276, 0, 0, 279, + 0, 281, 282, 283, 284, 285, 286, 0, 0, 287, + 0, 289, 0, -524, 291, 292, 293, 294, 295, 296, + 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 320, 0, 322, 323, 324, 325, 326, + 0, 327, 328, 0, 330, 0, 331, 332, 333, 334, + 335, 336, -524, 337, 338, 0, 0, 339, 340, 341, + 0, -524, 342, 343, 344, 0, 346, 0, 348, 349, + 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, + 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, + 366, 367, 368, 369, 0, 370, 371, 372, 373, 374, + 375, 0, 376, 377, 378, 379, 380, 381, 382, 383, + 0, 384, 385, 386, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 0, 397, 398, 0, 400, 401, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 415, 416, 0, 0, 417, 418, 419, + 420, 421, 422, 423, 424, 425, 0, 0, 427, 428, + 429, 430, 0, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 526, 444, 445, 0, 0, + 446, 447, 0, 448, 0, 450, 451, 452, 453, 454, + 0, 455, 456, 457, 0, 0, 458, 459, 460, 461, + 462, 0, 463, 464, 465, 466, 467, 468, 469, 470, + -524, 0, 471, 472, 473, 0, 474, 475, 476, 477, + 0, 478, 479, 480, 481, 482, 483, 484, 0, 485, + 0, 487, 488, 489, 490, 491, 492, 493, 0, 0, + 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, + 502, 503, 504, 505, 506, 507, 508, 509, 510, 511, + 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 1209, 0, 939, 0, - 0, 1210, 123, 124, 0, 125, 126, 127, 1211, 129, - 130, 131, 132, 940, 1212, 941, 942, 0, 137, 138, - 139, 140, 141, 142, 943, 612, 143, 144, 145, 146, - 944, 945, 149, 0, 150, 151, 152, 153, 613, 0, - 1213, 0, 1214, 157, 158, 159, 160, 161, 1215, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 1216, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 947, 184, 185, 948, 187, 949, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 950, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 951, 216, 217, 218, 219, 220, 615, 952, 222, - 0, 223, 224, 953, 226, 0, 227, 0, 228, 1218, - 0, 1219, 231, 232, 1220, 1221, 235, 0, 236, 0, - 954, 955, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 1222, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 1223, 262, 263, - 264, 265, 266, 267, 956, 957, 0, 958, 0, 271, - 1224, 1225, 274, 1226, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 1227, 284, 1228, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 1229, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 960, 1230, 961, 318, - 319, 320, 321, 962, 322, 323, 1231, 325, 963, 617, - 327, 964, 329, 330, 331, 0, 332, 333, 0, 0, - 965, 335, 336, 0, 0, 337, 338, 339, 1232, 341, - 1233, 619, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 620, 1234, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 966, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 1235, - 394, 395, 396, 967, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 1236, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 1237, 420, 421, - 968, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 1238, 437, 622, 439, 0, - 440, 441, 0, 442, 1239, 444, 445, 446, 447, 448, - 0, 449, 969, 970, 0, 0, 452, 453, 623, 455, - 624, 971, 457, 458, 1240, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 972, 2223, 479, - 1242, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 973, 0, 0, 0, 0, 0, - 0, 974, 975, 976, 0, 0, 0, 0, 977, 0, - 978, 0, 0, 0, 0, 979, 980, 981, 982, 1207, - 933, 642, 934, 935, 936, 937, 938, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 1209, 0, 939, 0, 0, 1210, 123, - 124, 0, 125, 126, 127, 1211, 129, 130, 131, 132, - 940, 1212, 941, 942, 0, 137, 138, 139, 140, 141, - 142, 943, 612, 143, 144, 145, 146, 944, 945, 149, - 0, 150, 151, 152, 153, 613, 0, 1213, 0, 1214, - 157, 158, 159, 160, 161, 1215, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 1216, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 947, 184, - 185, 948, 187, 949, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 950, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 951, 216, - 217, 218, 219, 220, 615, 952, 222, 0, 223, 224, - 953, 226, 0, 227, 0, 228, 1218, 0, 1219, 231, - 232, 1220, 1221, 235, 0, 236, 0, 954, 955, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 1222, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 1223, 262, 263, 264, 265, 266, - 267, 956, 957, 0, 958, 0, 271, 1224, 1225, 274, - 1226, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 1227, 284, 1228, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 1229, 295, 296, 297, 298, 299, 300, 301, + 0, 1157, 0, 117, 118, 119, 120, 121, 122, 123, + 124, 0, 125, 126, 127, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 129, 130, 0, 131, 132, 133, + 0, 135, 136, 137, 138, 139, 0, 141, 142, 0, + 143, 144, 145, 146, 147, 148, 0, 0, 149, 150, + 151, 152, 153, 154, 155, 0, 156, 157, 158, 159, + 160, 0, 0, 0, 162, 163, 164, 165, 166, 167, + 0, 169, 170, 171, 0, 172, 173, 174, 175, 176, + 177, 0, 0, 179, 180, 181, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, 0, 194, + 0, 195, 196, 197, 198, 199, 200, 0, 0, 201, + 202, 203, 204, 0, 0, 205, 206, 207, 208, 209, + 0, 210, 211, 212, 0, 213, 214, 215, 0, 216, + 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, + 0, 227, 0, 228, 229, 230, 231, 0, 232, 0, + 233, 0, 0, 0, 236, 237, 524, 0, 240, 0, + 241, 0, 242, 243, 244, 245, 0, 246, 247, 248, + 249, 250, 251, 252, 0, 254, 255, 256, 257, 0, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 267, 268, 269, 270, 271, 272, 273, 274, 0, 275, + 0, 276, 0, 0, 279, 0, 281, 282, 283, 284, + 285, 286, 0, 0, 287, 0, 289, 0, 0, 291, + 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 960, 1230, 961, 318, 319, 320, 321, - 962, 322, 323, 1231, 325, 963, 617, 327, 964, 329, - 330, 331, 0, 332, 333, 0, 0, 965, 335, 336, - 0, 0, 337, 338, 339, 1232, 341, 1233, 619, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 620, 1234, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 966, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 1235, 394, 395, 396, - 967, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 1236, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 1237, 420, 421, 968, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 1238, 437, 622, 439, 0, 440, 441, 0, - 442, 1239, 444, 445, 446, 447, 448, 0, 449, 969, - 970, 0, 0, 452, 453, 623, 455, 624, 971, 457, - 458, 1240, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 972, 0, 479, 1242, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 973, 0, 0, 0, 0, 0, 0, 974, 975, - 976, 0, 0, 0, 0, 977, 0, 978, 2271, 0, - 0, 0, 979, 980, 981, 982, 110, 933, 642, 934, - 935, 936, 937, 938, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, -1086, 119, 120, 121, 0, 0, 0, - 0, -1086, 939, 0, 0, 122, 123, 124, 0, 125, - 126, 127, 128, 129, 130, 131, 132, 940, 134, 941, - 942, 0, 137, 138, 139, 140, 141, 142, 943, 612, - 143, 144, 145, 146, 944, 945, 149, 0, 150, 151, - 152, 153, 613, 0, 614, 0, 946, 157, 158, 159, - 160, 161, 162, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 947, 184, 185, 948, 187, - 949, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 950, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 951, 216, 217, 218, 219, - 220, 615, 952, 222, 0, 223, 224, 953, 226, 0, - 227, 0, 228, 229, 0, 230, 231, 232, 233, 234, - 235, 0, 236, 0, 954, 955, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 261, 262, 263, 264, 265, 266, 267, 956, 957, - 0, 958, 0, 271, 272, 273, 274, 275, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 283, 284, 285, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 959, + 312, 313, 314, 315, 316, 317, 318, 319, 320, 0, + 322, 323, 324, 325, 326, 0, 327, 328, 0, 330, + 0, 331, 332, 333, 334, 335, 336, 0, 337, 338, + 0, 0, 339, 340, 341, 0, 0, 342, 343, 344, + 0, 346, 0, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, 358, 359, 0, 0, 0, 0, 360, + 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, + 370, 371, 372, 373, 374, 375, 0, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 0, + 397, 398, 0, 400, 401, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, + 0, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 0, 0, 427, 428, 429, 430, 0, 431, 432, + 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, + 526, 444, 445, 0, 0, 446, 447, 0, 448, 0, + 450, 451, 452, 453, 454, 0, 455, 456, 457, 0, + 0, 458, 459, 460, 461, 462, 0, 463, 464, 465, + 466, 467, 468, 469, 470, 0, 0, 471, 472, 473, + 0, 474, 475, 476, 477, 0, 478, 479, 480, 481, + 482, 483, 484, 0, 485, 0, 487, 488, 489, 490, + 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, + 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, + 507, 508, 509, 510, 511, 512, 513, 514, 515, 964, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 2429, 3212, 0, 117, 118, + 119, 120, 121, 122, 123, 124, 0, 125, 126, 127, + 3, 4, 0, 560, 0, 0, 0, 0, 565, 129, + 130, 0, 131, 132, 133, 567, 135, 136, 137, 568, + 569, 570, 571, 572, 0, 143, 144, 145, 146, 147, + 148, 0, 0, 149, 150, 151, 152, 576, 577, 155, + 0, 156, 157, 158, 159, 579, 0, 581, 0, 583, + 163, 164, 165, 166, 167, 584, 169, 170, 171, 0, + 172, 173, 174, 175, 176, 177, 0, 587, 179, 180, + 181, 182, 183, 184, 185, 186, 187, 188, 589, 190, + 191, 590, 193, 0, 194, 0, 195, 196, 197, 198, + 199, 200, 14, 15, 201, 202, 203, 204, 0, 0, + 205, 206, 207, 208, 209, 0, 210, 211, 212, 0, + 213, 214, 215, 0, 216, 217, 218, 219, 600, 221, + 222, 223, 224, 225, 601, 0, 227, 0, 228, 229, + 604, 231, 0, 232, 0, 233, 607, 23, 609, 236, + 237, 610, 611, 240, 0, 241, 0, 614, 615, 244, + 245, 0, 246, 247, 248, 249, 250, 251, 252, 617, + 254, 255, 256, 257, 0, 258, 259, 260, 261, 262, + 263, 264, 0, 265, 620, 621, 268, 269, 270, 271, + 272, 622, 623, 0, 625, 0, 276, 627, 628, 279, + 629, 281, 282, 283, 284, 285, 286, 0, 0, 287, + 632, 289, 633, 0, 291, 292, 293, 294, 295, 296, + 297, 298, 635, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, + 317, 318, 319, 636, 637, 638, 323, 324, 325, 639, + 0, 327, 328, 641, 330, 0, 643, 332, 644, 334, + 335, 336, 0, 337, 338, 0, 0, 339, 340, 341, + 0, 0, 342, 343, 650, 651, 346, 652, 653, 349, + 350, 351, 352, 353, 354, 355, 356, 357, 358, 359, + 27, 28, 29, 0, 360, 361, 658, 659, 364, 365, + 660, 367, 368, 369, 0, 370, 371, 372, 373, 374, + 375, 0, 376, 377, 378, 663, 380, 381, 382, 383, + 0, 384, 385, 386, 387, 388, 389, 390, 391, 392, + 393, 394, 395, 396, 0, 397, 398, 666, 400, 401, + 402, 667, 404, 405, 406, 407, 408, 409, 410, 411, + 412, 413, 414, 415, 416, 34, 669, 417, 418, 419, + 420, 421, 422, 670, 424, 425, 36, 672, 427, 428, + 673, 430, 0, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 675, 444, 676, 38, 0, + 446, 447, 39, 448, 680, 450, 451, 452, 453, 454, + 0, 455, 682, 683, 0, 0, 458, 459, 686, 461, + 687, 0, 463, 464, 689, 466, 467, 468, 469, 470, + 0, 0, 471, 472, 473, 41, 474, 475, 476, 477, + 0, 478, 479, 480, 481, 482, 965, 695, 0, 485, + 697, 487, 488, 489, 490, 491, 492, 493, 0, 0, + 494, 0, 45, 495, 496, 497, 498, 499, 500, 702, + 703, 704, 705, 706, 707, 708, 709, 710, 711, 712, + 512, 513, 514, 515, 0, 116, 46, 549, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 47, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 128, 129, 130, 0, 131, 132, + 133, 134, 135, 136, 137, 138, 139, 140, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 785, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 786, 0, 787, 0, 162, 163, 164, 165, 166, + 167, 168, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 178, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 14, 15, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 788, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 234, 23, 235, 236, 237, 238, 239, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 277, 278, 279, 280, 281, 282, 283, + 284, 285, 286, 789, 0, 287, 288, 289, 290, 0, + 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, 0, 327, 328, 329, + 330, 0, 790, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 791, 339, 340, 341, 0, 0, 342, 343, + 344, 345, 346, 347, 792, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 27, 28, 29, 0, + 360, 361, 793, 363, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 399, 400, 401, 402, 403, 794, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 34, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 36, 426, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, 444, 795, 38, 0, 446, 447, 39, 448, + 449, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 796, 461, 797, 0, 463, 464, + 798, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 41, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 799, 484, 0, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 45, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 0, 116, 46, 549, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 800, 0, 0, 0, + 117, 118, 119, 120, 121, 122, 123, 124, 0, 125, + 126, 127, 0, 0, 0, 0, 0, 0, 0, 0, + 128, 129, 130, 0, 131, 132, 133, 134, 135, 136, + 137, 138, 139, 140, 141, 142, 0, 143, 144, 145, + 146, 147, 148, 0, 785, 149, 150, 151, 152, 153, + 154, 155, 0, 156, 157, 158, 159, 786, 0, 787, + 0, 162, 163, 164, 165, 166, 167, 168, 169, 170, + 171, 0, 172, 173, 174, 175, 176, 177, 0, 178, + 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, 0, 194, 0, 195, 196, + 197, 198, 199, 200, 0, 0, 201, 202, 203, 204, + 0, 0, 205, 206, 207, 208, 209, 0, 210, 211, + 212, 0, 213, 214, 215, 0, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 788, 0, 227, 0, + 228, 229, 230, 231, 0, 232, 0, 233, 234, 0, + 235, 236, 237, 238, 239, 240, 0, 241, 0, 242, + 243, 244, 245, 0, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 0, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 0, 275, 0, 276, 277, + 278, 279, 280, 281, 282, 283, 284, 285, 286, 789, + 0, 287, 288, 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 960, 316, 961, 318, 319, 320, 321, 962, 322, 323, - 324, 325, 963, 617, 327, 964, 329, 330, 331, 0, - 332, 333, 0, 0, 965, 335, 336, 0, 0, 337, - 338, 339, 340, 341, 342, 619, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 620, 357, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 966, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 393, 394, 395, 396, 967, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, 420, 421, 968, 423, -1086, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 622, 439, 0, 440, 441, 0, 442, 443, 444, - 445, 446, 447, 448, 0, 449, 969, 970, 0, 0, - 452, 453, 623, 455, 624, 971, 457, 458, 625, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 972, 0, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 973, 0, - 0, 0, 0, 0, 0, 974, 975, 976, 0, 0, - 0, 0, 977, 0, 978, 0, 0, 0, 0, 979, - 980, 981, 982, 1207, 933, 642, 934, 935, 936, 937, - 938, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 1209, 0, 939, - 0, 0, 1210, 123, 124, 0, 125, 126, 127, 1211, - 129, 130, 131, 132, 940, 1212, 941, 942, 0, 137, - 138, 139, 140, 141, 142, 943, 612, 143, 144, 145, - 146, 944, 945, 149, 0, 150, 151, 152, 153, 613, - 0, 1213, 0, 1214, 157, 158, 159, 160, 161, 1215, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 1216, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 947, 184, 185, 948, 187, 949, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 950, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 951, 216, 217, 218, 219, 220, 615, 952, - 222, 0, 223, 224, 953, 226, 0, 227, 0, 228, - 1218, 0, 1219, 231, 232, 1220, 1221, 235, 0, 236, - 0, 954, 955, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 1222, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 1223, 262, - 263, 264, 265, 266, 267, 956, 957, 0, 958, 0, - 271, 1224, 1225, 274, 1226, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 1227, 284, 1228, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 1229, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 960, 1230, 961, - 318, 319, 320, 321, 962, 322, 323, 1231, 325, 963, - 617, 327, 964, 329, 330, 331, 0, 332, 333, 0, - 0, 965, 335, 336, 0, 0, 337, 338, 339, 1232, - 341, 1233, 619, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 620, - 1234, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 966, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 1235, 394, 395, 396, 967, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 1236, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 1237, 420, - 421, 968, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 1238, 437, 622, 439, - 0, 440, 441, 0, 442, 1239, 444, 445, 446, 447, - 448, 0, 449, 969, 970, 0, 0, 452, 453, 623, - 455, 624, 971, 457, 458, 1240, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 972, 0, - 479, 1242, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 973, 0, 0, 0, 0, - 0, 0, 974, 975, 976, 0, 0, 0, 0, 977, - 0, 978, 2995, 0, 0, 0, 979, 980, 981, 982, - 1207, 933, 642, 934, 935, 936, 937, 938, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 1209, 0, 939, 0, 0, 1210, - 123, 124, 0, 125, 126, 127, 1211, 129, 130, 131, - 132, 940, 1212, 941, 942, 0, 137, 138, 139, 140, - 141, 142, 943, 612, 143, 144, 145, 146, 944, 945, - 149, 0, 150, 151, 152, 153, 613, 0, 1213, 0, - 1214, 157, 158, 159, 160, 161, 1215, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 1216, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 947, - 184, 185, 948, 187, 949, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 950, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 951, - 216, 217, 218, 219, 220, 615, 952, 222, 0, 223, - 224, 953, 226, 0, 227, 0, 228, 1218, 0, 1219, - 231, 232, 1220, 1221, 235, 0, 236, 0, 954, 955, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 1222, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 1223, 262, 263, 264, 265, - 266, 267, 956, 957, 0, 958, 0, 271, 1224, 1225, - 274, 1226, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 1227, 284, 1228, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 1229, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 960, 1230, 961, 318, 319, 320, - 321, 962, 322, 323, 1231, 325, 963, 617, 327, 964, - 329, 330, 331, 0, 332, 333, 0, 0, 965, 335, - 336, 0, 0, 337, 338, 339, 1232, 341, 1233, 619, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 620, 1234, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 966, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 1235, 394, 395, - 396, 967, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 1236, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 1237, 420, 421, 968, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 1238, 437, 622, 439, 0, 440, 441, - 0, 442, 1239, 444, 445, 446, 447, 448, 0, 449, - 969, 970, 0, 0, 452, 453, 623, 455, 624, 971, - 457, 458, 1240, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 972, 0, 479, 1242, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 973, 0, 0, 0, 0, 0, 0, 974, - 975, 976, 0, 0, 0, 0, 977, 0, 978, 0, - 0, 0, 0, 979, 980, 981, 982, 110, 933, 642, - 934, 935, 0, 937, 938, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 939, 0, 0, 122, 123, 124, 0, - 125, 126, 127, 128, 129, 130, 131, 132, 940, 134, - 941, 942, 0, 137, 138, 139, 140, 141, 142, 943, - 612, 143, 144, 145, 146, 944, 945, 149, 0, 150, - 151, 152, 153, 613, 0, 614, 0, 156, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 947, 184, 185, 948, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 950, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 951, 216, 217, 218, - 219, 220, 615, 952, 222, 0, 223, 224, 953, 226, - 0, 227, 0, 228, 229, 21, 230, 231, 232, 233, - 234, 235, 0, 236, 0, 954, 955, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 261, 262, 263, 264, 265, 266, 267, 956, - 957, 0, 958, 0, 271, 272, 273, 274, 275, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 283, 284, - 285, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 959, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 960, 316, 961, 318, 319, 320, 321, 0, 322, - 323, 324, 325, 963, 617, 327, 964, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 340, 341, 342, 619, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 25, 26, 27, - 0, 354, 355, 620, 357, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 966, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 393, 394, 395, 396, 967, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 32, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 419, 420, 421, 968, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 622, 439, 0, 440, 441, 37, 442, 443, - 444, 445, 446, 447, 448, 0, 449, 969, 970, 0, - 0, 452, 453, 623, 455, 624, 971, 457, 458, 625, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 39, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 626, 972, 0, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 43, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 0, - 0, 44, 0, 0, 0, 110, 933, 642, 934, 935, - 936, 937, 938, 977, 0, 978, 0, 0, 0, 0, - 979, 980, 981, 982, 111, 112, 113, 114, 115, 116, - 117, 118, 1644, 119, 120, 121, 0, 0, 0, 0, - 0, 939, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 940, 134, 941, 942, - 0, 137, 138, 139, 140, 141, 142, 943, 612, 143, - 144, 145, 146, 944, 945, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 946, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 947, 184, 185, 948, 187, 949, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 951, 216, 217, 218, 219, 220, - 615, 952, 222, 0, 223, 224, 953, 226, 0, 227, - 0, 228, 229, 0, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 954, 955, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 956, 957, 0, - 958, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 283, 284, 285, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 959, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 960, - 316, 961, 318, 319, 320, 321, 962, 322, 323, 324, - 325, 963, 617, 327, 964, 329, 330, 331, 0, 332, - 333, 0, 0, 965, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, + 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 0, 327, 328, 329, 330, 0, 790, 332, + 333, 334, 335, 336, 0, 337, 338, 0, 791, 339, + 340, 341, 0, 0, 342, 343, 344, 345, 346, 347, + 792, 349, 350, 351, 352, 353, 354, 355, 356, 357, + 358, 359, 0, 0, 0, 0, 360, 361, 793, 363, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 966, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 967, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 968, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 969, 970, 0, 0, 452, - 453, 623, 455, 624, 971, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 972, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 973, 0, 0, - 0, 0, 0, 0, 974, 975, 976, 0, 0, 0, - 0, 977, 0, 978, 0, 0, 0, 0, 979, 980, - 981, 982, 110, 1659, 642, 934, 935, 936, 1660, 938, + 373, 374, 375, 0, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 0, 397, 398, 399, + 400, 401, 402, 403, 794, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 0, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 0, 426, + 427, 428, 429, 430, 0, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, 444, 795, + 0, 0, 446, 447, 0, 448, 449, 450, 451, 452, + 453, 454, 0, 455, 456, 457, 0, 0, 458, 459, + 796, 461, 797, 0, 463, 464, 798, 466, 467, 468, + 469, 470, 0, 0, 471, 472, 473, 0, 474, 475, + 476, 477, 0, 478, 479, 480, 481, 482, 799, 484, + 0, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, + 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, + 510, 511, 512, 513, 514, 515, 116, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 1661, - 119, 120, 121, 0, 0, 0, 0, 0, 939, 0, - 0, 122, 123, 124, 0, 125, 126, 127, 128, 129, - 130, 131, 132, 940, 134, 941, 942, 0, 137, 138, - 139, 140, 141, 142, 943, 612, 143, 144, 145, 146, - 944, 945, 149, 0, 150, 151, 152, 153, 613, 0, - 614, 0, 946, 157, 158, 159, 160, 161, 162, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 947, 184, 185, 948, 187, 949, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 950, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 951, 216, 217, 218, 219, 220, 615, 952, 222, - 0, 223, 224, 953, 226, 0, 227, 0, 228, 229, - 0, 230, 231, 232, 233, 234, 235, 0, 236, 0, - 954, 955, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 261, 262, 263, - 264, 265, 266, 267, 956, 957, 0, 958, 0, 271, - 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 283, 284, 285, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 959, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 960, 316, 961, 318, - 319, 320, 321, 962, 322, 323, 324, 325, 963, 617, - 327, 964, 329, 330, 331, 0, 332, 333, 0, 0, - 965, 335, 336, 0, 0, 337, 338, 339, 340, 341, - 342, 619, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 620, 357, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 966, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 393, - 394, 395, 396, 967, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, - 968, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 622, 439, 0, - 440, 441, 0, 442, 443, 444, 445, 446, 447, 448, - 0, 449, 969, 970, 0, 0, 452, 453, 623, 455, - 624, 971, 457, 458, 625, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 972, 0, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 973, 0, 0, 0, 0, 0, - 0, 974, 975, 976, 0, 0, 0, 0, 977, 0, - 978, 0, 0, 0, 0, 979, 980, 981, 982, 110, - 933, 642, 934, 935, 936, 937, 938, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 939, 0, 0, 122, 123, - 124, 0, 125, 126, 127, 128, 129, 130, 131, 132, - 940, 134, 941, 942, 0, 137, 138, 139, 140, 141, - 142, 943, 612, 143, 144, 145, 146, 944, 945, 149, - 0, 150, 151, 152, 153, 613, 0, 614, 0, 946, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 0, + 0, 0, 800, 0, 0, 117, 118, 119, 120, 121, + 122, 123, 124, 0, 125, 126, 127, 0, 0, 0, + 0, 0, 0, 0, 0, 128, 129, 130, 0, 131, + 132, 133, 134, 135, 136, 137, 138, 139, 140, 141, + 142, 0, 143, 144, 145, 146, 147, 148, 0, 785, + 149, 150, 151, 152, 153, 154, 155, 0, 156, 157, + 158, 159, 786, 0, 787, 0, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 947, 184, - 185, 948, 187, 949, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 950, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 951, 216, - 217, 218, 219, 220, 615, 952, 222, 0, 223, 224, - 953, 226, 0, 227, 0, 228, 229, 1379, 230, 231, - 232, 233, 234, 235, 0, 236, 0, 954, 955, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 261, 262, 263, 264, 265, 266, - 267, 956, 957, 0, 958, 0, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 283, 284, 285, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 959, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 960, 316, 961, 318, 319, 320, 321, - 962, 322, 323, 324, 325, 963, 617, 327, 964, 329, - 330, 331, 0, 332, 333, 0, 0, 965, 335, 336, - 0, 0, 337, 338, 339, 340, 341, 342, 619, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 620, 357, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 966, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 393, 394, 395, 396, - 967, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 968, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 622, 439, 0, 440, 441, 0, - 442, 443, 444, 445, 446, 447, 448, 0, 449, 969, - 970, 0, 0, 452, 453, 623, 455, 624, 971, 457, - 458, 625, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 972, 0, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 973, 0, 0, 0, 0, 0, 0, 974, 975, - 976, 0, 0, 0, 0, 977, 0, 978, 0, 0, - 0, 0, 979, 980, 981, 982, 110, 933, 642, 934, - 935, 936, 937, 938, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 939, 0, 0, 122, 123, 124, 0, 125, - 126, 127, 128, 129, 130, 131, 132, 940, 134, 941, - 942, 0, 137, 138, 139, 140, 141, 142, 943, 612, - 143, 144, 145, 146, 944, 945, 149, 0, 150, 151, - 152, 153, 613, 0, 614, 0, 946, 157, 158, 159, - 160, 161, 162, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 947, 184, 185, 948, 187, - 949, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 950, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 951, 216, 217, 218, 219, - 220, 615, 952, 222, 0, 223, 224, 953, 226, 0, - 227, 0, 228, 229, 0, 230, 231, 232, 233, 234, - 235, 0, 236, 0, 954, 955, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 261, 262, 263, 264, 265, 266, 267, 956, 957, - 0, 958, 0, 271, 272, 273, 274, 275, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 283, 284, 285, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 959, + 175, 176, 177, 0, 178, 179, 180, 181, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + 0, 194, 0, 195, 196, 197, 198, 199, 200, 0, + 0, 201, 202, 203, 204, 0, 0, 205, 206, 207, + 208, 209, 0, 210, 211, 212, 0, 213, 214, 215, + 0, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 788, 0, 227, 0, 228, 229, 230, 231, 0, + 232, 0, 233, 234, 0, 235, 236, 237, 238, 239, + 240, 0, 241, 0, 242, 243, 244, 245, 0, 246, + 247, 248, 249, 250, 251, 252, 253, 254, 255, 256, + 257, 0, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 266, 267, 268, 269, 270, 271, 272, 273, 274, + 0, 275, 0, 276, 277, 278, 279, 280, 281, 282, + 283, 284, 285, 286, 0, 0, 287, 288, 289, 290, + 0, 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, 0, 327, 328, + 329, 330, 0, 790, 332, 333, 334, 335, 336, 0, + 337, 338, 0, 0, 339, 340, 341, 0, 0, 342, + 343, 344, 345, 346, 347, 792, 349, 350, 351, 352, + 353, 354, 355, 356, 357, 358, 359, 0, 0, 0, + 0, 360, 361, 793, 363, 364, 365, 366, 367, 368, + 369, 0, 370, 371, 372, 373, 374, 375, 0, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 0, 397, 398, 399, 400, 401, 402, 403, 794, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 415, 416, 0, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 0, 426, 427, 428, 429, 430, 0, + 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, + 441, 442, 443, 444, 795, 0, 0, 446, 447, 0, + 448, 449, 450, 451, 452, 453, 454, 0, 455, 456, + 457, 0, 0, 458, 459, 796, 461, 797, 0, 463, + 464, 798, 466, 467, 468, 469, 470, 0, 0, 471, + 472, 473, 0, 474, 475, 476, 477, 0, 478, 479, + 480, 481, 482, 483, 484, 0, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 0, 0, 494, 0, 0, + 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, + 505, 506, 507, 508, 509, 510, 511, 512, 513, 514, + 515, 116, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 47, 0, 0, + 117, 118, 119, 120, 121, 122, 123, 124, 0, 125, + 126, 127, 0, 0, 0, 0, 0, 0, 0, 0, + 128, 129, 130, 0, 131, 132, 133, 134, 135, 136, + 137, 138, 139, 140, 141, 142, 0, 143, 144, 145, + 146, 147, 148, 0, 785, 149, 150, 151, 152, 153, + 154, 155, 0, 156, 157, 158, 159, 786, 0, 787, + 0, 162, 163, 164, 165, 166, 167, 168, 169, 170, + 171, 0, 172, 173, 174, 175, 176, 177, 0, 178, + 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, 0, 194, 0, 195, 196, + 197, 198, 199, 200, 0, 0, 201, 202, 203, 204, + 0, 0, 205, 206, 207, 208, 209, 0, 210, 211, + 212, 0, 213, 214, 215, 0, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 788, 0, 227, 0, + 228, 229, 230, 231, 0, 232, 0, 233, 234, 0, + 235, 236, 237, 238, 239, 240, 0, 241, 0, 242, + 243, 244, 245, 0, 246, 247, 248, 249, 250, 251, + 252, 253, 254, 255, 256, 257, 0, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 266, 267, 268, 269, + 270, 271, 272, 273, 274, 0, 275, 0, 276, 277, + 278, 279, 280, 281, 282, 283, 284, 285, 286, 0, + 0, 287, 288, 289, 290, 0, 291, 292, 293, 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 960, 316, 961, 318, 319, 320, 321, 962, 322, 323, - 324, 325, 963, 617, 327, 964, 329, 330, 331, 0, - 332, 333, 0, 0, 965, 335, 336, 0, 0, 337, - 338, 339, 340, 341, 342, 619, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 620, 357, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 966, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 393, 394, 395, 396, 967, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, 420, 421, 968, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 622, 439, 0, 440, 441, 0, 442, 443, 444, - 445, 446, 447, 448, 0, 449, 969, 970, 0, 0, - 452, 453, 623, 455, 624, 971, 457, 458, 625, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 972, 0, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 973, 0, - 0, 0, 0, 0, 0, 974, 975, 976, 0, 0, - 0, 0, 977, 0, 978, 1997, 0, 0, 0, 979, - 980, 981, 982, 110, 933, 642, 934, 935, 936, 937, - 938, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 939, - 0, 0, 122, 123, 124, 0, 125, 126, 127, 128, - 129, 130, 131, 132, 940, 134, 941, 942, 0, 137, - 138, 139, 140, 141, 142, 943, 612, 143, 144, 145, - 146, 944, 945, 149, 0, 150, 151, 152, 153, 613, - 0, 614, 0, 946, 157, 158, 159, 160, 161, 162, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 947, 184, 185, 948, 187, 949, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 950, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 951, 216, 217, 218, 219, 220, 615, 952, - 222, 0, 223, 224, 953, 226, 0, 227, 0, 228, - 229, 0, 230, 231, 232, 233, 234, 235, 0, 236, - 0, 954, 955, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 261, 262, - 263, 264, 265, 266, 267, 956, 957, 0, 958, 0, - 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 283, 284, 285, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 959, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 960, 316, 961, - 318, 319, 320, 321, 962, 322, 323, 324, 325, 963, - 617, 327, 964, 329, 330, 331, 0, 332, 333, 0, - 0, 965, 335, 336, 0, 0, 337, 338, 339, 340, - 341, 342, 619, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 620, - 357, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 966, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 393, 394, 395, 396, 967, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 0, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, - 421, 968, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 622, 439, - 0, 440, 441, 2609, 442, 443, 444, 445, 446, 447, - 448, 0, 449, 969, 970, 0, 0, 452, 453, 623, - 455, 624, 971, 457, 458, 625, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 972, 0, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 973, 0, 0, 0, 0, - 0, 0, 974, 975, 976, 0, 0, 0, 0, 977, - 0, 978, 0, 0, 0, 0, 979, 980, 981, 982, - 110, 933, 642, 934, 935, 936, 937, 938, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 939, 0, 0, 122, - 123, 124, 0, 125, 126, 127, 128, 129, 130, 131, - 132, 940, 134, 941, 942, 0, 137, 138, 139, 140, - 141, 142, 943, 612, 143, 144, 145, 146, 944, 945, - 149, 0, 150, 151, 152, 153, 613, 0, 614, 0, - 946, 157, 158, 159, 160, 161, 162, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 947, - 184, 185, 948, 187, 949, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 950, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 951, - 216, 217, 218, 219, 220, 615, 952, 222, 0, 223, - 224, 953, 226, 0, 227, 0, 228, 229, 0, 230, - 231, 232, 233, 234, 235, 0, 236, 0, 954, 955, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 261, 262, 263, 264, 265, - 266, 267, 956, 957, 0, 958, 0, 271, 272, 273, - 274, 275, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 283, 284, 285, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 959, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 960, 316, 961, 318, 319, 320, - 321, 962, 322, 323, 324, 325, 963, 617, 327, 964, - 329, 330, 331, 0, 332, 333, 0, 0, 965, 335, - 336, 0, 0, 337, 338, 339, 340, 341, 342, 619, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 620, 357, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 966, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 393, 394, 395, - 396, 967, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 419, 420, 421, 968, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 622, 439, 0, 440, 441, - 0, 442, 443, 444, 445, 446, 447, 448, 0, 449, - 969, 970, 0, 0, 452, 453, 623, 455, 624, 971, - 457, 458, 625, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 972, 0, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 0, 2727, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 973, 0, 0, 0, 0, 0, 0, 974, - 975, 976, 0, 0, 0, 0, 977, 0, 978, 0, - 0, 0, 0, 979, 980, 981, 982, 110, 933, 642, - 934, 935, 936, 937, 938, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 2938, 119, 120, 121, 0, 0, - 0, 0, 0, 939, 0, 0, 122, 123, 124, 0, - 125, 126, 127, 128, 129, 130, 131, 132, 940, 134, - 941, 942, 0, 137, 138, 139, 140, 141, 142, 943, - 612, 143, 144, 145, 146, 944, 945, 149, 0, 150, - 151, 152, 153, 613, 0, 614, 0, 946, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 172, 173, 2939, 175, 176, - 177, 178, 179, 180, 181, 182, 947, 184, 185, 948, - 187, 949, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 950, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 951, 216, 217, 218, - 219, 220, 615, 952, 222, 0, 223, 224, 953, 226, - 0, 227, 0, 228, 229, 0, 230, 231, 232, 233, - 234, 235, 0, 236, 0, 2940, 955, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 261, 262, 263, 264, 265, 266, 267, 956, - 957, 0, 958, 0, 271, 272, 273, 274, 275, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 283, 284, - 285, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 959, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 960, 316, 961, 318, 319, 320, 321, 962, 322, - 323, 324, 325, 963, 617, 327, 964, 329, 330, 331, - 0, 332, 333, 0, 0, 965, 335, 336, 0, 0, - 337, 338, 339, 340, 341, 342, 619, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 620, 357, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 966, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 393, 394, 395, 2941, 967, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 419, 420, 421, 968, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 622, 439, 0, 440, 441, 0, 442, 443, - 444, 445, 446, 447, 448, 0, 449, 969, 970, 0, - 0, 452, 453, 623, 455, 624, 971, 457, 458, 625, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 972, 0, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 973, - 0, 0, 0, 0, 0, 0, 974, 975, 976, 0, - 0, 0, 0, 977, 0, 2942, 0, 0, 0, 0, - 979, 980, 981, 982, 110, 933, 642, 934, 935, 936, - 937, 938, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 939, 0, 0, 122, 123, 124, 0, 125, 126, 127, - 128, 129, 130, 131, 132, 940, 134, 941, 942, 0, - 137, 138, 139, 140, 141, 142, 943, 612, 143, 144, - 145, 146, 944, 945, 149, 0, 150, 151, 152, 153, - 613, 0, 614, 0, 946, 157, 158, 159, 160, 161, - 162, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 172, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 947, 184, 185, 948, 187, 949, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 950, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 951, 216, 217, 218, 219, 220, 615, - 952, 222, 0, 223, 224, 953, 226, 0, 227, 0, - 228, 229, 0, 230, 231, 232, 233, 234, 235, 0, - 236, 0, 954, 955, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 261, - 262, 263, 264, 265, 266, 267, 956, 957, 0, 958, - 0, 271, 272, 273, 274, 275, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 283, 284, 285, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 959, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 960, 316, - 961, 318, 319, 320, 321, 962, 322, 323, 324, 325, - 963, 617, 327, 964, 329, 330, 331, 0, 332, 333, - 0, 0, 965, 335, 336, 0, 0, 337, 338, 339, - 340, 341, 342, 619, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 620, 357, 358, 359, 360, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 966, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 393, 394, 395, 396, 967, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, - 420, 421, 968, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 622, - 439, 0, 440, 441, 3089, 442, 443, 444, 445, 446, - 447, 448, 0, 449, 969, 970, 0, 0, 452, 453, - 623, 455, 624, 971, 457, 458, 625, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 972, - 0, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 973, 0, 0, 0, - 0, 0, 0, 974, 975, 976, 0, 0, 0, 0, - 977, 0, 978, 0, 0, 0, 0, 979, 980, 981, - 982, 110, 933, 642, 934, 935, 936, 937, 938, 0, + 315, 316, 317, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 0, 327, 328, 329, 330, 0, 790, 332, + 333, 334, 335, 336, 0, 337, 338, 0, 0, 339, + 340, 341, 0, 0, 342, 343, 344, 345, 346, 347, + 792, 349, 350, 351, 352, 353, 354, 355, 356, 357, + 358, 359, 0, 0, 0, 0, 360, 361, 793, 363, + 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, + 373, 374, 375, 0, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 0, 397, 398, 399, + 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 0, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 0, 426, + 427, 428, 429, 430, 0, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, 444, 795, + 0, 0, 446, 447, 0, 448, 449, 450, 451, 452, + 453, 454, 0, 455, 456, 457, 0, 0, 458, 459, + 796, 461, 797, 0, 463, 464, 798, 466, 467, 468, + 469, 470, 0, 0, 471, 472, 473, 0, 474, 475, + 476, 477, 0, 478, 479, 480, 481, 482, 483, 484, + 0, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, + 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, + 510, 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 0, 0, 939, 0, 0, - 122, 123, 124, 0, 125, 126, 127, 128, 129, 130, - 131, 132, 940, 134, 941, 942, 0, 137, 138, 139, - 140, 141, 142, 943, 612, 143, 144, 145, 146, 944, - 945, 149, 0, 150, 151, 152, 153, 613, 0, 614, - 0, 946, 157, 158, 159, 160, 161, 162, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 947, 184, 185, 948, 187, 949, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 950, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 951, 216, 217, 218, 219, 220, 615, 952, 222, 0, - 223, 224, 953, 226, 0, 227, 0, 228, 229, 0, - 230, 231, 232, 233, 234, 235, 0, 236, 0, 954, - 955, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 261, 262, 263, 264, - 265, 266, 267, 956, 957, 0, 958, 0, 271, 272, - 273, 274, 275, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 283, 284, 285, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 959, 295, 296, 297, 298, 299, + 0, 0, 3300, 0, 0, 117, 118, 119, 120, 121, + 122, 123, 124, 0, 125, 126, 127, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 129, 130, 0, 131, + 132, 133, 0, 135, 136, 137, 138, 139, 0, 141, + 142, 0, 143, 144, 145, 146, 147, 148, 0, 0, + 149, 150, 151, 152, 153, 154, 155, 0, 156, 157, + 158, 159, 160, 0, 0, 0, 162, 163, 164, 165, + 166, 167, 0, 169, 170, 171, 0, 172, 173, 174, + 175, 176, 177, 0, 0, 179, 180, 181, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + 0, 194, 0, 195, 196, 197, 198, 199, 200, 14, + 15, 201, 202, 203, 204, 0, 0, 205, 206, 207, + 208, 209, 0, 210, 211, 212, 0, 213, 214, 215, + 0, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, 0, 227, 0, 228, 229, 230, 231, 0, + 232, 0, 233, 0, 23, 0, 236, 237, 524, 0, + 240, 0, 241, 0, 242, 243, 244, 245, 0, 246, + 247, 248, 249, 250, 251, 252, 0, 254, 255, 256, + 257, 0, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 267, 268, 269, 270, 271, 272, 273, 274, + 0, 275, 0, 276, 0, 0, 279, 0, 281, 282, + 283, 284, 285, 286, 0, 0, 287, 0, 289, 0, + 0, 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 960, 316, 961, 318, 319, - 320, 321, 962, 322, 323, 324, 325, 963, 617, 327, - 964, 329, 330, 331, 0, 332, 333, 0, 0, 965, - 335, 336, 0, 0, 337, 338, 339, 340, 341, 342, - 619, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 0, 0, 0, 0, 354, 355, 620, 357, 358, - 359, 360, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 966, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 393, 394, - 395, 396, 967, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 0, 0, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 419, 420, 421, 968, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 622, 439, 0, 440, - 441, 3225, 442, 443, 444, 445, 446, 447, 448, 0, - 449, 969, 970, 0, 0, 452, 453, 623, 455, 624, - 971, 457, 458, 625, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 0, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 477, 972, 0, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 0, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, - 507, 508, 509, 973, 0, 0, 0, 0, 0, 0, - 974, 975, 976, 0, 0, 0, 0, 977, 0, 978, - 0, 0, 0, 0, 979, 980, 981, 982, 110, 933, - 642, 934, 935, 936, 937, 938, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 939, 0, 0, 122, 123, 124, - 0, 125, 126, 127, 128, 129, 130, 131, 132, 940, - 134, 941, 942, 0, 137, 138, 139, 140, 141, 142, - 943, 612, 143, 144, 145, 146, 944, 945, 149, 0, - 150, 151, 152, 153, 613, 0, 614, 0, 946, 157, - 158, 159, 160, 161, 162, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 947, 184, 185, - 948, 187, 949, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 950, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 951, 216, 217, - 218, 219, 220, 615, 952, 222, 0, 223, 224, 953, - 226, 0, 227, 0, 228, 229, 0, 230, 231, 232, - 233, 234, 235, 0, 236, 0, 954, 955, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 261, 262, 263, 264, 265, 266, 267, - 956, 957, 0, 958, 0, 271, 272, 273, 274, 275, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 283, - 284, 285, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 959, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 960, 316, 961, 318, 319, 320, 321, 962, - 322, 323, 324, 325, 963, 617, 327, 964, 329, 330, - 331, 0, 332, 333, 0, 0, 965, 335, 336, 0, - 0, 337, 338, 339, 340, 341, 342, 619, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 620, 357, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 966, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 393, 394, 395, 396, 967, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 419, 420, 421, 968, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 622, 439, 0, 440, 441, 0, 442, - 443, 444, 445, 446, 447, 448, 0, 449, 969, 970, - 0, 0, 452, 453, 623, 455, 624, 971, 457, 458, - 625, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 972, 0, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 973, 0, 0, 0, 0, 0, 0, 974, 975, 976, - 0, 0, 0, 0, 977, 0, 978, 0, 0, 0, - 0, 979, 980, 981, 982, 110, 933, 642, 934, 935, - 936, 937, 938, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 939, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 940, 134, 941, 942, - 0, 137, 138, 139, 140, 141, 142, 943, 612, 143, - 144, 145, 146, 944, 945, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 946, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 947, 184, 185, 948, 187, 949, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 951, 216, 217, 218, 219, 220, - 615, 952, 222, 0, 223, 224, 953, 226, 0, 227, - 0, 228, 229, 0, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 954, 955, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 956, 957, 0, - 958, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 283, 284, 285, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 959, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 960, - 316, 961, 318, 319, 320, 321, 962, 322, 323, 324, - 325, 963, 617, 327, 964, 329, 330, 331, 0, 332, - 333, 0, 0, 965, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, + 310, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, 0, 322, 323, 324, 325, 326, 0, 327, 328, + 0, 330, 0, 331, 332, 333, 334, 335, 336, 0, + 337, 338, 0, 0, 339, 340, 341, 0, 0, 342, + 343, 344, 0, 346, 0, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, 358, 359, 27, 28, 29, + 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, + 369, 0, 370, 371, 372, 373, 374, 375, 0, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 0, 397, 398, 0, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 415, 416, 34, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 36, 0, 427, 428, 429, 430, 0, + 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, + 441, 442, 526, 444, 445, 38, 0, 446, 447, 39, + 448, 0, 450, 451, 452, 453, 454, 0, 455, 456, + 457, 0, 0, 458, 459, 460, 461, 462, 0, 463, + 464, 465, 466, 467, 468, 469, 470, 0, 0, 471, + 472, 473, 41, 474, 475, 476, 477, 0, 478, 479, + 480, 481, 482, 799, 484, 0, 485, 0, 487, 488, + 489, 490, 491, 492, 493, 0, 0, 494, 0, 45, + 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, + 505, 506, 507, 508, 509, 510, 511, 512, 513, 514, + 515, 523, 0, 46, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 47, 0, 0, + 117, 118, 119, 120, 121, 122, 123, 124, 886, 125, + 126, 127, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 129, 130, 0, 131, 132, 133, 0, 135, 136, + 137, 138, 139, 0, 141, 142, 0, 143, 144, 145, + 146, 147, 148, 0, 0, 149, 150, 151, 152, 153, + 154, 155, 0, 156, 157, 158, 159, 160, 0, 0, + 0, 162, 163, 164, 165, 166, 167, 0, 169, 170, + 171, 0, 172, 173, 174, 175, 176, 177, 0, 0, + 179, 180, 181, 182, 183, 184, 185, 186, 187, 188, + 189, 190, 191, 192, 193, 0, 194, 0, 195, 196, + 197, 198, 199, 200, 0, 0, 201, 202, 203, 204, + 0, 0, 205, 206, 207, 208, 209, 0, 210, 211, + 212, 0, 213, 214, 215, 0, 216, 217, 218, 219, + 220, 221, 222, 223, 224, 225, 226, 0, 227, 0, + 228, 229, 230, 231, 0, 232, 0, 233, 0, 23, + 0, 236, 237, 524, 0, 240, 0, 241, 0, 242, + 243, 244, 245, 0, 246, 247, 248, 249, 250, 251, + 252, 0, 254, 255, 256, 257, 0, 258, 259, 260, + 261, 262, 263, 264, 0, 265, 0, 267, 268, 269, + 270, 271, 272, 273, 274, 0, 275, 0, 276, 0, + 0, 279, 0, 281, 282, 283, 284, 285, 286, 0, + 0, 287, 0, 289, 0, 0, 291, 292, 293, 294, + 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, + 315, 316, 317, 318, 319, 320, 0, 322, 323, 324, + 325, 326, 0, 327, 328, 0, 330, 0, 331, 332, + 333, 334, 335, 336, 0, 337, 338, 0, 0, 339, + 340, 341, 0, 0, 342, 343, 344, 0, 346, 0, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + 358, 359, 27, 28, 29, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 966, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 967, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 968, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 969, 970, 0, 0, 452, - 453, 623, 455, 624, 971, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 972, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 973, 0, 0, - 0, 0, 0, 0, 1664, 1665, 976, 0, 0, 0, - 0, 977, 0, 978, 0, 0, 0, 0, 979, 980, - 981, 982, 110, 2134, 642, 934, 935, 936, 937, 938, + 373, 374, 375, 0, 376, 377, 378, 379, 380, 381, + 382, 383, 0, 384, 385, 386, 387, 388, 389, 390, + 391, 392, 393, 394, 395, 396, 0, 397, 398, 0, + 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, 412, 413, 414, 415, 416, 34, 0, 417, + 418, 419, 420, 421, 422, 423, 424, 425, 0, 0, + 427, 428, 429, 430, 0, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 526, 444, 445, + 0, 0, 446, 447, 39, 448, 0, 450, 451, 452, + 453, 454, 0, 455, 887, 457, 0, 0, 888, 459, + 460, 461, 462, 0, 463, 464, 465, 466, 467, 468, + 469, 470, 0, 0, 471, 472, 473, 41, 474, 475, + 476, 477, 0, 478, 479, 480, 481, 482, 799, 484, + 0, 485, 0, 487, 488, 489, 490, 491, 492, 493, + 0, 0, 494, 0, 45, 495, 496, 497, 498, 499, + 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, + 510, 511, 512, 513, 514, 515, 523, 0, 46, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 939, 0, - 0, 122, 123, 124, 0, 125, 126, 127, 128, 129, - 130, 131, 132, 940, 134, 941, 942, 0, 137, 138, - 139, 140, 141, 142, 943, 612, 143, 144, 145, 146, - 944, 945, 149, 0, 150, 151, 152, 153, 613, 0, - 614, 0, 946, 157, 158, 159, 160, 161, 162, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 947, 184, 185, 948, 187, 949, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 950, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 951, 216, 217, 218, 219, 220, 615, 952, 222, - 0, 223, 224, 953, 226, 0, 227, 0, 228, 229, - 0, 230, 231, 232, 233, 234, 235, 0, 236, 0, - 954, 955, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 261, 262, 263, - 264, 265, 266, 267, 956, 957, 0, 958, 0, 271, - 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 283, 284, 285, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 959, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 960, 316, 961, 318, - 319, 320, 321, 962, 322, 323, 324, 325, 963, 617, - 327, 964, 329, 330, 331, 0, 332, 333, 0, 0, - 965, 335, 336, 0, 0, 337, 338, 339, 340, 341, - 342, 619, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 620, 357, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 966, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 393, - 394, 395, 396, 967, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, - 968, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 622, 439, 0, - 440, 441, 0, 442, 443, 444, 445, 446, 447, 448, - 0, 449, 969, 970, 0, 0, 452, 453, 623, 455, - 624, 971, 457, 458, 625, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 972, 0, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 973, 0, 0, 0, 0, 0, - 0, 974, 975, 976, 0, 0, 0, 0, 977, 0, - 978, 0, 0, 0, 0, 979, 980, 981, 982, 110, - 933, 642, 934, 935, 936, 937, 938, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 939, 0, 0, 122, 123, - 124, 0, 125, 126, 127, 128, 129, 130, 131, 132, - 940, 134, 941, 942, 0, 137, 138, 139, 140, 141, - 142, 943, 612, 143, 144, 145, 146, 944, 945, 149, - 0, 150, 151, 152, 153, 613, 0, 614, 0, 946, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 947, 184, - 185, 948, 187, 949, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 950, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 951, 216, - 217, 218, 219, 220, 615, 952, 222, 0, 223, 224, - 953, 226, 0, 227, 0, 228, 229, 0, 230, 231, - 232, 233, 234, 235, 0, 236, 0, 954, 955, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 261, 262, 263, 264, 265, 266, - 267, 956, 957, 0, 958, 0, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 283, 284, 285, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 959, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 960, 316, 961, 318, 319, 320, 321, - 962, 322, 323, 324, 325, 963, 617, 327, 964, 329, - 330, 331, 0, 332, 333, 0, 0, 965, 335, 336, - 0, 0, 337, 338, 339, 340, 341, 342, 619, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 620, 357, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 966, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 393, 394, 395, 396, - 967, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 968, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 622, 439, 0, 440, 441, 0, - 442, 443, 444, 445, 446, 447, 448, 0, 449, 969, - 970, 0, 0, 452, 453, 623, 455, 624, 971, 457, - 458, 625, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 972, 0, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 973, 0, 0, 0, 0, 0, 0, 974, 2217, - 976, 0, 0, 0, 0, 977, 0, 978, 0, 0, - 0, 0, 979, 980, 981, 982, 110, 933, 642, 934, - 935, 936, 937, 938, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 939, 0, 0, 122, 123, 124, 0, 125, - 126, 127, 128, 129, 130, 131, 132, 940, 134, 941, - 942, 0, 137, 138, 139, 140, 141, 142, 943, 612, - 143, 144, 145, 146, 944, 945, 149, 0, 150, 151, - 152, 153, 613, 0, 614, 0, 946, 157, 158, 159, - 160, 161, 162, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 172, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 947, 184, 185, 948, 187, - 949, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 950, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 951, 216, 217, 218, 219, - 220, 615, 952, 222, 0, 223, 224, 953, 226, 0, - 227, 0, 228, 229, 0, 230, 231, 232, 233, 234, - 235, 0, 236, 0, 954, 955, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 248, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 261, 262, 263, 264, 265, 266, 267, 956, 957, - 0, 958, 0, 271, 272, 273, 274, 275, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 283, 284, 285, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 959, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 960, 316, 961, 318, 319, 320, 321, 962, 322, 323, - 324, 325, 963, 617, 327, 964, 329, 330, 331, 0, - 332, 333, 0, 0, 965, 335, 336, 0, 0, 337, - 338, 339, 340, 341, 342, 619, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 620, 357, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 966, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 393, 394, 395, 396, 967, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, 420, 421, 968, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 622, 439, 0, 440, 441, 0, 442, 443, 444, - 445, 446, 447, 448, 0, 449, 969, 970, 0, 0, - 452, 453, 623, 455, 624, 971, 457, 458, 625, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 972, 0, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 973, 0, - 0, 0, 0, 0, 0, 974, 975, 976, 0, 0, - 0, 0, 977, 0, 2418, 0, 0, 0, 0, 979, - 980, 981, 982, 110, 933, 642, 934, 935, 936, 937, - 938, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 939, - 0, 0, 122, 123, 124, 0, 125, 126, 127, 128, - 129, 130, 131, 132, 940, 134, 941, 942, 0, 137, - 138, 139, 140, 141, 142, 943, 612, 143, 144, 145, - 146, 944, 945, 149, 0, 150, 151, 152, 153, 613, - 0, 614, 0, 946, 157, 158, 159, 160, 161, 162, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 947, 184, 185, 948, 187, 949, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 950, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 951, 216, 217, 218, 219, 220, 615, 952, - 222, 0, 223, 224, 953, 226, 0, 227, 0, 228, - 229, 0, 230, 231, 232, 233, 234, 235, 0, 236, - 0, 954, 955, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 261, 262, - 263, 264, 265, 266, 267, 956, 957, 0, 958, 0, - 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 283, 284, 285, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 959, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 960, 316, 961, - 318, 319, 320, 321, 962, 322, 323, 324, 325, 963, - 617, 327, 964, 329, 330, 331, 0, 332, 333, 0, - 0, 965, 335, 336, 0, 0, 337, 338, 339, 340, - 341, 342, 619, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 620, - 357, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 966, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 393, 394, 395, 396, 967, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 0, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, - 421, 968, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 622, 439, - 0, 440, 441, 0, 442, 443, 444, 445, 446, 447, - 448, 0, 449, 969, 970, 0, 0, 452, 453, 623, - 455, 624, 971, 457, 458, 625, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 972, 0, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 973, 0, 0, 0, 0, - 0, 0, 974, 2993, 976, 0, 0, 0, 0, 977, - 0, 978, 0, 0, 0, 0, 979, 980, 981, 982, - 110, 933, 642, 934, 935, 936, 937, 938, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 939, 0, 0, 122, - 123, 124, 0, 125, 126, 127, 128, 129, 130, 131, - 132, 940, 134, 941, 942, 0, 137, 138, 139, 140, - 141, 142, 943, 612, 143, 144, 145, 146, 944, 945, - 149, 0, 150, 151, 152, 153, 613, 0, 614, 0, - 946, 157, 158, 159, 160, 161, 162, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 172, 173, - 2939, 175, 176, 177, 178, 179, 180, 181, 182, 947, - 184, 185, 948, 187, 949, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 950, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 951, - 216, 217, 218, 219, 220, 615, 952, 222, 0, 223, - 224, 953, 226, 0, 227, 0, 228, 229, 0, 230, - 231, 232, 233, 234, 235, 0, 236, 0, 2940, 955, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 261, 262, 263, 264, 265, - 266, 267, 956, 957, 0, 958, 0, 271, 272, 273, - 274, 275, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 283, 284, 285, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 959, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 960, 316, 961, 318, 319, 320, - 321, 962, 322, 323, 324, 325, 963, 617, 327, 964, - 329, 330, 331, 0, 332, 333, 0, 0, 965, 335, - 336, 0, 0, 337, 338, 339, 340, 341, 342, 619, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 620, 357, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 966, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 393, 394, 395, - 2941, 967, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 419, 420, 421, 968, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 622, 439, 0, 440, 441, - 0, 442, 443, 444, 445, 446, 447, 448, 0, 449, - 969, 970, 0, 0, 452, 453, 623, 455, 624, 971, - 457, 458, 625, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 972, 0, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 973, 0, 0, 0, 0, 0, 0, 974, - 975, 976, 0, 0, 0, 0, 977, 0, 2942, 0, - 0, 0, 0, 979, 980, 981, 982, 110, 933, 642, - 934, 935, 936, 937, 938, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 939, 0, 0, 122, 123, 124, 0, - 125, 126, 127, 128, 129, 130, 131, 3399, 940, 134, - 941, 942, 0, 137, 138, 139, 140, 141, 142, 943, - 612, 143, 144, 145, 146, 944, 945, 149, 0, 150, - 151, 152, 153, 613, 0, 614, 0, 946, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 172, 173, 174, 3400, 176, - 177, 178, 179, 180, 181, 182, 947, 184, 185, 948, - 187, 949, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 950, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 951, 216, 217, 218, - 219, 220, 615, 952, 222, 0, 223, 224, 953, 226, - 0, 227, 0, 228, 229, 0, 230, 231, 232, 233, - 234, 235, 0, 236, 0, 954, 955, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 261, 262, 263, 264, 265, 266, 267, 956, - 957, 0, 958, 0, 271, 272, 273, 274, 275, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 283, 284, - 285, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 959, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 960, 316, 961, 318, 319, 320, 321, 962, 322, - 323, 324, 325, 963, 617, 327, 964, 329, 330, 331, - 0, 332, 333, 0, 0, 965, 335, 336, 0, 0, - 337, 338, 339, 340, 341, 342, 619, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 620, 357, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 966, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 393, 394, 395, 396, 967, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 419, 420, 421, 968, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 622, 439, 0, 440, 441, 0, 442, 443, - 444, 445, 446, 447, 448, 0, 449, 969, 970, 0, - 0, 452, 453, 623, 455, 624, 971, 457, 458, 625, - 460, 461, 3401, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 972, 0, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 973, - 0, 0, 0, 0, 0, 0, 974, 975, 976, 0, - 0, 0, 0, 977, 0, 978, 0, 0, 0, 0, - 979, 980, 981, 982, 110, 933, 642, 934, 935, 936, - 937, 938, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 939, 0, 0, 122, 123, 124, 0, 125, 126, 127, - 128, 129, 130, 131, 132, 940, 134, 941, 942, 0, - 137, 138, 139, 140, 141, 142, 943, 612, 143, 144, - 145, 146, 944, 945, 149, 0, 150, 151, 152, 153, - 613, 0, 614, 0, 946, 157, 158, 159, 160, 161, - 162, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 172, 173, 174, 3400, 176, 177, 178, 179, - 180, 181, 182, 947, 184, 185, 948, 187, 949, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 950, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 951, 216, 217, 218, 219, 220, 615, - 952, 222, 0, 223, 224, 953, 226, 0, 227, 0, - 228, 229, 0, 230, 231, 232, 233, 234, 235, 0, - 236, 0, 954, 955, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 248, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 261, - 262, 263, 264, 265, 266, 267, 956, 957, 0, 958, - 0, 271, 272, 273, 274, 275, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 283, 284, 285, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 959, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 960, 316, - 961, 318, 319, 320, 321, 962, 322, 323, 324, 325, - 963, 617, 327, 964, 329, 330, 331, 0, 332, 333, - 0, 0, 965, 335, 336, 0, 0, 337, 338, 339, - 340, 341, 342, 619, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 620, 357, 358, 359, 360, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 966, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 393, 394, 395, 396, 967, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 419, - 420, 421, 968, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 622, - 439, 0, 440, 441, 0, 442, 443, 444, 445, 446, - 447, 448, 0, 449, 969, 970, 0, 0, 452, 453, - 623, 455, 624, 971, 457, 458, 625, 460, 461, 3401, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 972, - 0, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 973, 0, 0, 0, - 0, 0, 0, 974, 975, 976, 0, 0, 0, 0, - 977, 0, 978, 0, 0, 0, 0, 979, 980, 981, - 982, 110, 933, 642, 934, 935, 936, 937, 938, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 0, 0, 939, 0, 0, - 122, 123, 124, 0, 125, 126, 127, 128, 129, 130, - 131, 132, 940, 134, 941, 942, 0, 137, 138, 139, - 140, 141, 142, 943, 612, 143, 144, 145, 146, 944, - 945, 149, 0, 150, 151, 152, 153, 613, 0, 614, - 0, 156, 157, 158, 159, 160, 161, 162, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 172, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 947, 184, 185, 948, 187, 0, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 950, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 951, 216, 217, 218, 219, 220, 615, 952, 222, 0, - 223, 224, 953, 226, 0, 227, 0, 228, 229, 0, - 230, 231, 232, 233, 234, 235, 0, 236, 0, 954, - 955, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 248, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 261, 262, 263, 264, - 265, 266, 267, 956, 957, 0, 958, 0, 271, 272, - 273, 274, 275, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 283, 284, 285, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 959, 295, 296, 297, 298, 299, + 0, 0, 47, 0, 0, 117, 118, 119, 120, 121, + 122, 123, 124, 0, 125, 126, 127, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 129, 130, 0, 131, + 132, 133, 0, 135, 136, 137, 138, 139, 0, 141, + 142, 0, 143, 144, 145, 146, 147, 148, 0, 0, + 149, 150, 151, 152, 153, 154, 155, 0, 156, 157, + 158, 159, 160, 0, 0, 0, 162, 163, 164, 165, + 166, 167, 0, 169, 170, 171, 0, 172, 173, 174, + 175, 176, 177, 0, 0, 179, 180, 181, 182, 183, + 184, 185, 186, 187, 188, 189, 190, 191, 192, 193, + 0, 194, 0, 195, 196, 197, 198, 199, 200, 0, + 0, 201, 202, 203, 204, 0, 0, 205, 206, 207, + 208, 209, 0, 210, 211, 212, 0, 213, 214, 215, + 0, 216, 217, 218, 219, 220, 221, 222, 223, 224, + 225, 226, 0, 227, 0, 228, 229, 230, 231, 0, + 232, 0, 233, 0, 23, 0, 236, 237, 524, 0, + 240, 0, 241, 0, 242, 243, 244, 245, 0, 246, + 247, 248, 249, 250, 251, 252, 0, 254, 255, 256, + 257, 0, 258, 259, 260, 261, 262, 263, 264, 0, + 265, 0, 267, 268, 269, 270, 271, 272, 273, 274, + 0, 275, 0, 276, 0, 0, 279, 0, 281, 282, + 283, 284, 285, 286, 0, 0, 287, 0, 289, 0, + 0, 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 960, 316, 961, 318, 319, - 320, 321, 0, 322, 323, 324, 325, 963, 617, 327, - 964, 329, 330, 331, 0, 332, 333, 0, 0, 965, - 335, 336, 0, 0, 337, 338, 339, 340, 341, 342, - 619, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 0, 0, 0, 0, 354, 355, 620, 357, 358, - 359, 360, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 966, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 393, 394, - 395, 396, 967, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 0, 0, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 419, 420, 421, 968, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 622, 439, 0, 440, - 441, 0, 442, 443, 444, 445, 446, 447, 448, 0, - 449, 969, 970, 0, 0, 452, 453, 623, 455, 624, - 971, 457, 458, 625, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 0, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 477, 972, 0, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 0, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, - 507, 508, 509, 0, 0, 0, 0, 0, 0, 0, - 1365, 1366, 0, 0, 0, 0, 0, 977, 0, 978, - 0, 0, 0, 0, 979, 980, 981, 982, 110, 933, - 642, 934, 935, 936, 937, 938, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 939, 0, 0, 122, 123, 124, - 0, 125, 126, 127, 128, 129, 130, 131, -2003, 940, - 134, 941, 942, 0, 137, 138, 139, 140, 141, 142, - 943, 612, 143, 144, 145, 146, 944, 945, 149, 0, - 150, 151, 152, 153, 613, 0, 614, 0, 946, 157, - 158, 159, 160, 161, 162, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 172, 173, 174, 3400, - 176, 177, 178, 179, 180, 181, 182, 947, 184, 185, - 948, 187, 949, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 950, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 951, 216, 217, - 218, 219, 220, 615, 952, 222, 0, 223, 224, 953, - 226, 0, 227, 0, 228, 229, 0, 230, 231, 232, - 233, -2003, 235, 0, 236, 0, 954, 955, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, -2003, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 261, 262, 263, 264, 265, 266, 267, - 956, 957, 0, 958, 0, 271, 0, 0, 274, 275, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 283, - 284, -2003, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 959, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 960, 316, 961, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 963, 617, 327, 964, 329, 330, - 331, 0, 332, 333, 0, 0, 965, 335, 336, 0, - 0, 337, 338, 339, 340, 341, 342, 619, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 620, 357, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 966, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 393, 394, 395, 396, 967, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, -2003, 420, 421, 968, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 622, 439, 0, 440, 441, 0, 442, - 443, 444, 445, 446, 447, 448, 0, 449, 969, 970, - 0, 0, 452, 453, 623, 455, 624, 971, 457, 458, - 625, 460, 461, 3401, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 972, 0, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - -2003, 0, 0, 0, 0, 0, 0, 974, 975, 976, - 0, 0, 0, 0, 977, 0, 978, 0, 0, 0, - 0, 979, 980, 981, 982, 110, 933, 642, 934, 935, - 0, 937, 938, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 939, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 940, 134, 941, 942, - 0, 137, 138, 139, 140, 141, 142, 943, 612, 143, - 144, 145, 146, 944, 945, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 156, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 947, 184, 185, 948, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 951, 216, 217, 218, 219, 220, - 615, 952, 222, 0, 223, 224, 953, 226, 0, 227, - 0, 228, 229, 0, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 954, 955, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 956, 957, 0, - 958, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 283, 284, 285, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 959, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 960, - 316, 961, 318, 319, 320, 321, 0, 322, 323, 324, - 325, 963, 617, 327, 964, 329, 330, 331, 0, 332, - 333, 0, 0, 334, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 966, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 2120, 2121, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 968, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 969, 970, 0, 0, 452, - 453, 623, 455, 624, 971, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 972, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 0, 0, 0, - 0, 0, 0, 0, 2122, 2123, 0, 0, 0, 0, - 0, 977, 0, 978, 0, 0, 0, 0, 979, 980, - 981, 982, 110, 933, 642, 934, 935, 936, 937, 938, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 939, 0, - 0, 122, 123, 124, 0, 125, 126, 127, 128, 129, - 130, 131, 132, 940, 134, 941, 942, 0, 137, 138, - 139, 140, 141, 142, 943, 612, 143, 144, 145, 146, - 944, 945, 149, 0, 150, 151, 152, 153, 613, 0, - 614, 0, 156, 157, 158, 159, 160, 161, 162, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 947, 184, 185, 948, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 950, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 951, 216, 217, 218, 219, 220, 615, 952, 222, - 0, 223, 224, 953, 226, 0, 227, 0, 228, 229, - 0, 230, 231, 232, 233, 234, 235, 0, 236, 0, - 954, 955, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 261, 262, 263, - 264, 265, 266, 267, 956, 957, 0, 958, 0, 271, - 0, 273, 274, 275, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 283, 284, 285, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 959, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 960, 316, 961, 318, - 319, 320, 321, 0, 322, 323, 324, 325, 963, 617, - 327, 964, 329, 330, 331, 0, 332, 333, 0, 0, - 965, 335, 336, 0, 0, 337, 338, 339, 340, 341, - 342, 619, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 620, 357, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 966, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 393, - 394, 395, 396, 967, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, - 968, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 622, 439, 0, - 440, 441, 0, 442, 443, 444, 445, 446, 447, 448, - 0, 449, 969, 970, 0, 0, 452, 453, 623, 455, - 624, 971, 457, 458, 625, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 972, 0, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 0, 0, 0, 0, 0, 0, - 0, 1365, 1366, 0, 0, 0, 0, 0, 977, 0, - 978, 0, 0, 0, 0, 979, 980, 981, 982, 110, - 933, 642, 934, 935, 0, 937, 938, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 939, 0, 0, 122, 123, - 124, 0, 125, 126, 127, 128, 129, 130, 131, 132, - 940, 134, 941, 942, 0, 137, 138, 139, 140, 141, - 142, 943, 612, 143, 144, 145, 146, 944, 945, 149, - 0, 150, 151, 152, 153, 613, 0, 614, 0, 156, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 947, 184, - 185, 948, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 950, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 951, 216, - 217, 218, 219, 220, 615, 952, 222, 0, 223, 224, - 953, 226, 0, 227, 0, 228, 229, 0, 230, 231, - 232, 233, 234, 235, 0, 236, 0, 954, 955, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 261, 262, 263, 264, 265, 266, - 267, 956, 957, 0, 958, 0, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 283, 284, 285, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 959, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 960, 316, 961, 318, 319, 320, 321, - 0, 322, 323, 324, 325, 963, 617, 327, 964, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 340, 341, 342, 619, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 620, 357, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 966, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 393, 394, 395, 396, - 967, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 968, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 622, 439, 0, 440, 441, 0, - 442, 443, 444, 445, 446, 447, 448, 0, 449, 969, - 970, 0, 0, 452, 453, 623, 455, 624, 971, 457, - 458, 625, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 972, 0, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 0, 0, 0, 0, 0, 0, 110, 933, 642, - 934, 935, 936, 937, 938, 977, 0, 978, 0, 0, - 0, 0, 979, 980, 981, 982, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 939, 0, 0, 122, 123, 124, 0, - 125, 126, 127, 128, 129, 130, 131, 0, 940, 134, - 941, 942, 0, 137, 138, 139, 140, 141, 142, 943, - 612, 143, 144, 145, 146, 944, 945, 149, 0, 150, - 151, 152, 153, 613, 0, 614, 0, 946, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 947, 184, 185, 948, - 187, 949, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 950, 203, 204, 0, 205, 206, 207, 0, 208, 0, - 210, 0, 211, 212, 213, 214, 951, 216, 217, 218, - 219, 220, 615, 952, 222, 0, 223, 224, 953, 226, - 0, 227, 0, 228, 229, 0, 230, 231, 232, 233, - 0, 235, 0, 236, 0, 954, 955, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 0, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 261, 262, 263, 264, 265, 266, 267, 956, - 957, 0, 958, 0, 271, 0, 0, 274, 275, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 283, 284, - 0, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 959, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 960, 316, 961, 318, 319, 320, 321, 0, 322, - 323, 0, 325, 963, 617, 327, 964, 329, 330, 331, - 0, 332, 333, 0, 0, 965, 335, 336, 0, 0, - 337, 338, 339, 340, 341, 342, 619, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 620, 357, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 966, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 393, 394, 395, 396, 967, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 0, 420, 421, 968, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 622, 439, 0, 440, 441, 0, 442, 443, - 444, 445, 446, 447, 448, 0, 449, 969, 970, 0, - 0, 452, 453, 623, 455, 624, 971, 457, 458, 625, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 972, 0, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 0, - 0, 0, 0, 0, 0, 0, 974, 975, 976, 0, - 775, 1296, 642, 977, 0, 978, 937, 0, 0, 0, - 979, 980, 981, 982, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 776, 0, 0, 0, 0, 777, - 123, 124, 0, 125, 126, 127, 778, 129, 130, 131, - 779, 780, 781, 782, 783, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 784, 785, - 149, 0, 150, 151, 152, 153, 786, 0, 787, 0, - 788, 157, 158, 159, 160, 161, 789, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 790, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 791, - 184, 185, 792, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 793, - 216, 217, 218, 219, 220, 794, 1297, 222, 0, 223, - 224, 795, 226, 0, 227, 0, 228, 796, 0, 797, - 231, 232, 798, 799, 235, 0, 236, 0, 800, 801, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 802, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 803, 804, 263, 264, 265, - 266, 267, 805, 806, 0, 807, 0, 271, 808, 809, - 274, 810, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 811, 284, 812, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 813, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 814, 815, 816, 318, 319, 320, - 817, 0, 322, 323, 818, 325, 0, 819, 327, 820, - 329, 330, 331, 0, 332, 333, 1298, 0, 334, 335, - 336, 0, 0, 337, 338, 821, 822, 341, 823, 824, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 825, 826, 358, 359, - 827, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 828, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 829, 394, 395, - 396, 830, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 831, 409, 410, 411, 412, 413, - 414, 832, 416, 417, 418, 833, 420, 421, 834, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 835, 437, 836, 439, 0, 440, 441, - 0, 442, 837, 444, 445, 446, 447, 448, 0, 449, - 838, 839, 0, 0, 452, 453, 840, 455, 841, 1299, - 457, 458, 842, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 1195, 844, 0, 479, 845, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 846, 847, 848, - 849, 850, 851, 852, 853, 854, 855, 856, 506, 507, - 508, 509, 0, 0, 0, 0, 0, 0, 0, 1300, - 1301, 2284, 0, 110, 933, 642, 934, 935, 2285, 937, - 0, 0, 0, 0, 0, 0, 982, 0, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 939, - 0, 0, 122, 123, 124, 0, 125, 126, 127, 128, - 129, 130, 131, 132, 940, 134, 941, 942, 0, 137, - 138, 139, 140, 141, 142, 943, 0, 143, 144, 145, - 146, 944, 945, 149, 0, 150, 151, 152, 153, 154, - 0, 155, 0, 156, 157, 158, 159, 160, 161, 162, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 172, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 947, 184, 185, 948, 187, 0, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 950, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 951, 216, 217, 218, 219, 220, 221, 952, - 222, 0, 223, 224, 953, 226, 0, 227, 0, 228, - 229, 0, 230, 231, 232, 233, 234, 235, 0, 236, - 2959, 954, 955, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 248, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 261, 262, - 263, 264, 265, 266, 267, 956, 957, 0, 958, 0, - 271, 272, 273, 274, 275, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 283, 284, 285, 0, 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, 960, 316, 961, - 318, 319, 320, 321, 0, 322, 323, 324, 325, 963, - 326, 327, 964, 329, 330, 331, 0, 332, 333, 0, - 0, 334, 335, 336, 0, 0, 337, 338, 339, 340, - 341, 342, 343, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 356, - 357, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 966, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 0, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, - 421, 968, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 0, 440, 441, 0, 442, 443, 444, 445, 446, 447, - 448, 0, 449, 969, 970, 0, 0, 452, 453, 454, - 455, 456, 971, 457, 458, 459, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 972, 0, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, + 310, 311, 312, 313, 314, 315, 316, 317, 318, 319, + 320, 0, 322, 323, 324, 325, 326, 0, 327, 328, + 0, 330, 0, 331, 332, 333, 334, 335, 336, 0, + 337, 338, 0, 0, 339, 340, 341, 0, 0, 342, + 343, 344, 0, 346, 0, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, 358, 359, 27, 28, 29, + 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, + 369, 0, 370, 371, 372, 373, 374, 375, 0, 376, + 377, 378, 379, 380, 381, 382, 383, 0, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + 396, 0, 397, 398, 0, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, 412, 413, 414, + 415, 416, 34, 0, 417, 418, 419, 420, 421, 422, + 423, 424, 425, 0, 0, 427, 428, 429, 430, 0, + 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, + 441, 442, 526, 444, 445, 0, 0, 446, 447, 39, + 448, 0, 450, 451, 452, 453, 454, 0, 455, 456, + 457, 0, 0, 458, 459, 460, 461, 462, 0, 463, + 464, 465, 466, 467, 468, 469, 470, 0, 0, 471, + 472, 473, 41, 474, 475, 476, 477, 0, 478, 479, + 480, 481, 482, 799, 484, 0, 485, 0, 487, 488, + 489, 490, 491, 492, 493, 0, 0, 494, 0, 45, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 0, 0, 0, 0, 110, - 933, 642, 934, 935, 0, 937, 0, 0, 0, 977, - 0, 2656, 0, 0, 0, 0, 979, 980, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 939, 0, 0, 122, 123, - 124, 0, 125, 126, 127, 128, 129, 130, 131, 132, - 940, 134, 941, 942, 0, 137, 138, 139, 140, 141, - 142, 943, 0, 143, 144, 145, 146, 944, 945, 149, - 0, 150, 151, 152, 153, 154, 0, 155, 0, 156, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 947, 184, - 185, 948, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 950, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 951, 216, - 217, 218, 219, 220, 221, 952, 222, 0, 223, 224, - 953, 226, 0, 227, 0, 228, 229, 0, 230, 231, - 232, 233, 234, 235, 0, 236, 0, 954, 955, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 261, 262, 263, 264, 265, 266, - 267, 956, 957, 0, 958, 0, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 283, 284, 285, 0, 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, 960, 316, 961, 318, 319, 320, 321, - 0, 322, 323, 324, 325, 963, 326, 327, 964, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 340, 341, 342, 343, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 356, 357, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 966, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 968, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 0, 440, 441, 0, - 442, 443, 444, 445, 446, 447, 448, 0, 449, 969, - 970, 0, 0, 452, 453, 454, 455, 456, 971, 457, - 458, 459, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 972, 0, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 505, 506, 507, 508, 509, 510, 511, 512, 513, 514, + 515, 0, 523, 46, 549, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 47, 0, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, + 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 0, 0, 0, 0, 517, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 977, 0, 2656, 0, 0, - 0, 0, 979, 980, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 0, 891, 0, 0, 123, 124, 0, 125, 126, - 127, 0, 129, 130, 131, 132, 133, 0, 135, 136, - 0, 137, 138, 139, 140, 141, 142, 0, 0, 143, - 144, 145, 146, 147, 148, 149, 0, 150, 151, 152, - 153, 154, 0, 0, 0, 156, 157, 158, 159, 160, - 161, 0, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 0, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 183, 184, 185, 186, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 202, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, -504, - 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, - 221, 0, 222, -504, 223, 224, 225, 226, -504, 227, - 0, 228, 0, 0, 0, 231, 232, 518, 0, 235, - 0, 236, 0, 237, 238, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 0, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 0, 262, 263, 264, 265, 266, 267, 268, 269, -504, - 270, 0, 271, 0, 0, 274, 0, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 0, 284, 0, -504, - 286, 287, 288, 289, 290, 291, 292, 293, 519, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, - 0, 317, 318, 319, 320, 321, 0, 322, 323, 0, - 325, 0, 326, 327, 328, 329, 330, 331, -504, 332, - 333, 0, 0, 334, 335, 336, 0, -504, 337, 338, - 339, 0, 341, 0, 343, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 356, 0, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 373, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 0, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 0, 420, 421, 422, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 520, 437, - 438, 439, 0, 440, 441, 0, 442, 0, 444, 445, - 446, 447, 448, 0, 449, 450, 451, 0, 0, 452, - 453, 454, 455, 456, 0, 457, 458, 459, 460, 461, - 462, 463, 464, -504, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 478, 0, 479, 0, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 517, 0, 540, + 509, 510, 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1048, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 123, 124, 0, - 125, 126, 127, 0, 129, 130, 131, 132, 133, 0, - 135, 136, 0, 137, 138, 139, 140, 141, 142, 0, - 0, 143, 144, 145, 146, 147, 148, 149, 0, 150, - 151, 152, 153, 154, 0, 0, 0, 156, 157, 158, - 159, 160, 161, 0, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 0, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 202, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 221, 0, 222, 0, 223, 224, 225, 226, - 0, 227, 0, 228, 0, 0, 0, 231, 232, 518, - 0, 235, 0, 236, 0, 237, 238, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 0, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 0, 262, 263, 264, 265, 266, 267, 268, - 269, 0, 270, 0, 271, 0, 0, 274, 0, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 0, 284, - 0, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 519, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 0, 0, 0, 985, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 129, 130, 0, + 131, 132, 133, 0, 135, 136, 137, 138, 139, 0, + 141, 142, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 153, 154, 155, 0, 156, + 157, 158, 159, 160, 0, 0, 0, 162, 163, 164, + 165, 166, 167, 0, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 0, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 0, 227, 0, 228, 229, 230, 231, + 0, 232, 0, 233, 0, 0, 0, 236, 237, 524, + 0, 240, 0, 241, 0, 242, 243, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 0, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 267, 268, 269, 270, 271, 272, 273, + 274, 0, 275, 0, 276, 0, 0, 279, 0, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 0, 289, + 0, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, 0, 322, 323, 324, 325, 326, 0, 327, + 328, 0, 330, 0, 331, 332, 333, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 0, 346, 0, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 0, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 0, 427, 428, 429, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 526, 444, 445, 0, 0, 446, 447, + 0, 448, 0, 450, 451, 452, 453, 454, 0, 455, + 456, 457, 0, 0, 458, 459, 460, 461, 462, 0, + 463, 464, 465, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 484, 0, 485, 0, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1460, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 0, 317, 318, 319, 320, 321, 0, 322, - 323, 0, 325, 0, 326, 327, 328, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 0, 341, 0, 343, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 356, 0, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 373, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 0, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 0, 420, 421, 422, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 520, 437, 438, 439, 0, 440, 441, 0, 442, 0, - 444, 445, 446, 447, 448, 0, 449, 450, 451, 0, - 0, 452, 453, 454, 455, 456, 0, 457, 458, 459, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 478, 0, 479, 0, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 775, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, + 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, + 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, + 509, 510, 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 2371, 3142, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 3, 4, 0, 776, 0, 0, 0, 0, 777, 123, - 124, 0, 125, 126, 127, 778, 129, 130, 131, 779, - 780, 781, 782, 783, 0, 137, 138, 139, 140, 141, - 142, 0, 0, 143, 144, 145, 146, 784, 785, 149, - 0, 150, 151, 152, 153, 786, 0, 787, 0, 788, - 157, 158, 159, 160, 161, 789, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 790, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 791, 184, - 185, 792, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 202, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 793, 216, - 217, 218, 219, 220, 794, 0, 222, 0, 223, 224, - 795, 226, 0, 227, 0, 228, 796, 21, 797, 231, - 232, 798, 799, 235, 0, 236, 0, 800, 801, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 802, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 803, 804, 263, 264, 265, 266, - 267, 805, 806, 0, 807, 0, 271, 808, 809, 274, - 810, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 811, 284, 812, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 813, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 814, 815, 816, 318, 319, 320, 817, - 0, 322, 323, 818, 325, 0, 819, 327, 820, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 821, 822, 341, 823, 824, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 25, - 26, 27, 0, 354, 355, 825, 826, 358, 359, 827, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 828, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 829, 394, 395, 396, - 830, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 32, 831, 409, 410, 411, 412, 413, 414, - 832, 416, 417, 418, 833, 420, 421, 834, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 835, 437, 836, 439, 0, 440, 441, 37, - 442, 837, 444, 445, 446, 447, 448, 0, 449, 838, - 839, 0, 0, 452, 453, 840, 455, 841, 0, 457, - 458, 842, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 39, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 843, 844, 0, 479, 845, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 43, - 489, 490, 491, 492, 493, 494, 846, 847, 848, 849, - 850, 851, 852, 853, 854, 855, 856, 506, 507, 508, - 509, 0, 110, 44, 540, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 45, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 122, 123, 124, 0, 125, 126, 127, 128, 129, - 130, 131, 132, 133, 134, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 612, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 613, 0, - 614, 0, 156, 157, 158, 159, 160, 161, 162, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 172, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 615, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 229, - 21, 230, 231, 232, 233, 234, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 248, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 261, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 272, 273, 274, 275, 276, 277, 278, 279, 280, 281, - 616, 0, 282, 283, 284, 285, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 294, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 0, 0, 0, 2195, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 129, 130, 0, + 131, 132, 133, 0, 135, 136, 137, 138, 139, 0, + 141, 142, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 153, 154, 155, 0, 156, + 157, 158, 159, 160, 0, 0, 0, 162, 163, 164, + 165, 166, 167, 0, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 0, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 0, 227, 0, 228, 229, 230, 231, + 0, 232, 0, 233, 0, 0, 0, 236, 237, 524, + 0, 240, 0, 241, 0, 242, 243, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 0, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 267, 268, 269, 270, 271, 272, 273, + 274, 0, 275, 0, 276, 0, 0, 279, 0, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 0, 289, + 0, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, - 319, 320, 321, 0, 322, 323, 324, 325, 0, 617, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 618, - 334, 335, 336, 0, 0, 337, 338, 339, 340, 341, - 342, 619, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 25, 26, 27, 0, 354, 355, 620, 357, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 393, - 394, 395, 396, 397, 621, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 32, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 622, 439, 0, - 440, 441, 37, 442, 443, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 623, 455, - 624, 0, 457, 458, 625, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 39, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 626, 478, 0, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 43, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 0, 110, 44, 540, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 627, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 0, 0, 0, 122, 123, 124, 0, 125, 126, - 127, 128, 129, 130, 131, 132, 133, 134, 135, 136, - 0, 137, 138, 139, 140, 141, 142, 0, 612, 143, - 144, 145, 146, 147, 148, 149, 0, 150, 151, 152, - 153, 613, 0, 614, 0, 156, 157, 158, 159, 160, - 161, 162, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 172, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 183, 184, 185, 186, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 202, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, - 615, 0, 222, 0, 223, 224, 225, 226, 0, 227, - 0, 228, 229, 0, 230, 231, 232, 233, 234, 235, - 0, 236, 0, 237, 238, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 248, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 261, 262, 263, 264, 265, 266, 267, 268, 269, 0, - 270, 0, 271, 272, 273, 274, 275, 276, 277, 278, - 279, 280, 281, 616, 0, 282, 283, 284, 285, 0, - 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, 0, 322, 323, 324, - 325, 0, 617, 327, 328, 329, 330, 331, 0, 332, - 333, 0, 618, 334, 335, 336, 0, 0, 337, 338, - 339, 340, 341, 342, 619, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 620, 357, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 373, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 393, 394, 395, 396, 397, 621, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, 421, 422, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 622, 439, 0, 440, 441, 0, 442, 443, 444, 445, - 446, 447, 448, 0, 449, 450, 451, 0, 0, 452, - 453, 623, 455, 624, 0, 457, 458, 625, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 626, - 478, 0, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 110, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 627, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 0, 0, 0, 122, 123, 124, 0, - 125, 126, 127, 128, 129, 130, 131, 132, 133, 134, - 135, 136, 0, 137, 138, 139, 140, 141, 142, 0, - 612, 143, 144, 145, 146, 147, 148, 149, 0, 150, - 151, 152, 153, 613, 0, 614, 0, 156, 157, 158, - 159, 160, 161, 162, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 172, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 202, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 615, 0, 222, 0, 223, 224, 225, 226, - 0, 227, 0, 228, 229, 0, 230, 231, 232, 233, - 234, 235, 0, 236, 0, 237, 238, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 248, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 261, 262, 263, 264, 265, 266, 267, 268, - 269, 0, 270, 0, 271, 272, 273, 274, 275, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 283, 284, - 285, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 294, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 319, 320, 0, 322, 323, 324, 325, 326, 0, 327, + 328, 0, 330, 0, 331, 332, 333, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 0, 346, 0, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 0, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 0, 427, 428, 429, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 526, 444, 445, 0, 0, 446, 447, + 0, 448, 0, 450, 451, 452, 453, 454, 0, 455, + 456, 457, 0, 0, 458, 459, 460, 461, 462, 0, + 463, 464, 465, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 484, 0, 485, 0, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2429, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 316, 317, 318, 319, 320, 321, 0, 322, - 323, 324, 325, 0, 617, 327, 328, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 340, 341, 342, 619, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 620, 357, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 373, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 393, 394, 395, 396, 397, 621, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 419, 420, 421, 422, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 622, 439, 0, 440, 441, 0, 442, 443, - 444, 445, 446, 447, 448, 0, 449, 450, 451, 0, - 0, 452, 453, 623, 455, 624, 0, 457, 458, 625, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 478, 0, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 110, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 45, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 0, 0, 0, 122, 123, - 124, 0, 125, 126, 127, 128, 129, 130, 131, 132, - 133, 134, 135, 136, 0, 137, 138, 139, 140, 141, - 142, 0, 612, 143, 144, 145, 146, 147, 148, 149, - 0, 150, 151, 152, 153, 613, 0, 614, 0, 156, - 157, 158, 159, 160, 161, 162, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 202, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 215, 216, - 217, 218, 219, 220, 615, 0, 222, 0, 223, 224, - 225, 226, 0, 227, 0, 228, 229, 0, 230, 231, - 232, 233, 234, 235, 0, 236, 0, 237, 238, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 248, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 261, 262, 263, 264, 265, 266, - 267, 268, 269, 0, 270, 0, 271, 272, 273, 274, - 275, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 283, 284, 285, 0, 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, - 0, 322, 323, 324, 325, 0, 617, 327, 328, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 340, 341, 342, 619, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 620, 357, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 373, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 393, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 419, 420, 421, 422, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 622, 439, 0, 440, 441, 0, - 442, 443, 444, 445, 446, 447, 448, 0, 449, 450, - 451, 0, 0, 452, 453, 623, 455, 624, 0, 457, - 458, 625, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 478, 0, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, + 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 517, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 3228, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 712, 119, - 120, 121, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 123, 124, 0, 125, 126, 127, 0, 129, 130, - 131, 132, 133, 0, 135, 136, 0, 137, 138, 139, - 140, 141, 142, 0, 0, 143, 144, 145, 146, 147, - 148, 149, 0, 150, 151, 152, 153, 154, 0, 0, - 0, 156, 157, 158, 159, 160, 161, 0, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 0, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 183, 184, 185, 186, 187, 0, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 202, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 215, 216, 217, 218, 219, 220, 221, 0, 222, 0, - 223, 224, 225, 226, 0, 227, 0, 228, 0, 21, - 0, 231, 232, 518, 0, 235, 0, 236, 0, 237, - 238, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 0, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 0, 262, 263, 264, - 265, 266, 267, 268, 269, 0, 270, 0, 271, 0, - 0, 274, 0, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 0, 284, 0, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 519, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 315, 0, 317, 318, 319, - 320, 321, 0, 322, 323, 0, 325, 0, 326, 327, - 328, 329, 330, 331, 0, 332, 333, 0, 0, 334, - 335, 336, 0, 0, 337, 338, 339, 0, 341, 0, - 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 25, 26, 27, 0, 354, 355, 356, 0, 358, - 359, 360, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 373, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 0, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 32, 0, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 0, 420, 421, 422, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 520, 437, 438, 439, 0, 440, - 441, 37, 442, 0, 444, 445, 446, 447, 448, 0, - 449, 713, 451, 0, 0, 714, 453, 454, 455, 456, - 0, 457, 458, 459, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 39, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 626, 478, 0, 479, 0, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 43, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, - 507, 508, 509, 517, 0, 44, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 45, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 123, 124, 0, 125, 126, 127, 0, - 129, 130, 131, 132, 133, 0, 135, 136, 0, 137, - 138, 139, 140, 141, 142, 0, 0, 143, 144, 145, - 146, 147, 148, 149, 0, 150, 151, 152, 153, 154, - 0, 0, 0, 156, 157, 158, 159, 160, 161, 0, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 0, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 183, 184, 185, 186, 187, 0, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 202, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 215, 216, 217, 218, 219, 220, 221, 0, - 222, 0, 223, 224, 225, 226, 0, 227, 0, 228, - 0, 21, 0, 231, 232, 518, 0, 235, 0, 236, - 0, 237, 238, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 0, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 0, 262, - 263, 264, 265, 266, 267, 268, 269, 0, 270, 0, - 271, 0, 0, 274, 0, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 0, 284, 0, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 519, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 315, 0, 317, - 318, 319, 320, 321, 0, 322, 323, 0, 325, 0, - 326, 327, 328, 329, 330, 331, 0, 332, 333, 0, - 0, 334, 335, 336, 0, 0, 337, 338, 339, 0, - 341, 0, 343, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 25, 26, 27, 0, 354, 355, 356, - 0, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 373, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 0, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 32, 0, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 0, 420, - 421, 422, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 520, 437, 438, 439, - 0, 440, 441, 37, 442, 0, 444, 445, 446, 447, - 448, 0, 449, 450, 451, 0, 0, 452, 453, 454, - 455, 456, 0, 457, 458, 459, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 39, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 626, 478, 0, - 479, 0, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 43, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 0, 517, 44, 540, 0, + 509, 510, 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 45, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, + 0, 0, 0, 2560, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 129, 130, 0, + 131, 132, 133, 0, 135, 136, 137, 138, 139, 0, + 141, 142, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 153, 154, 155, 0, 156, + 157, 158, 159, 160, 0, 0, 0, 162, 163, 164, + 165, 166, 167, 0, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 0, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 0, 227, 0, 228, 229, 230, 231, + 0, 232, 0, 233, 0, 0, 0, 236, 237, 524, + 0, 240, 0, 241, 0, 242, 243, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 0, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 267, 268, 269, 270, 271, 272, 273, + 274, 0, 275, 0, 276, 0, 0, 279, 0, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 0, 289, + 0, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, 0, 322, 323, 324, 325, 326, 0, 327, + 328, 0, 330, 0, 331, 332, 333, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 0, 346, 0, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 0, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 0, 427, 428, 429, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 526, 444, 445, 0, 0, 446, 447, + 0, 448, 0, 450, 451, 452, 453, 454, 0, 455, + 456, 457, 0, 0, 458, 459, 460, 461, 462, 0, + 463, 464, 465, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 484, 0, 485, 0, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 3206, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 876, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 540, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1419, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 540, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2146, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 540, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 2371, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 450, 451, 0, 0, 452, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 0, 540, 0, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, + 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, + 509, 510, 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 2502, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, + 0, 0, 0, 2052, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 129, 130, 0, + 131, 132, 133, 0, 135, 136, 137, 138, 139, 0, + 141, 142, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 153, 154, 155, 0, 156, + 157, 158, 159, 160, 0, 0, 0, 162, 163, 164, + 165, 166, 167, 0, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 0, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 0, 227, 0, 228, 229, 230, 231, + 0, 232, 0, 233, 0, 0, 0, 236, 237, 524, + 0, 240, 0, 241, 0, 242, 243, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 0, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 267, 268, 269, 270, 271, 272, 273, + 274, 0, 275, 0, 276, 0, 0, 279, 0, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 0, 289, + 0, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, 0, 322, 323, 324, 325, 326, 0, 327, + 328, 0, 330, 0, 331, 332, 333, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 0, 346, 0, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 0, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 0, 427, 428, 429, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 526, 444, 445, 0, 0, 446, 447, + 0, 448, 0, 450, 451, 452, 453, 454, 0, 455, + 456, 457, 0, 0, 458, 459, 460, 461, 462, 0, + 463, 464, 465, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 484, 0, 485, 0, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 2155, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 3136, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 2003, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2106, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, - 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 0, 2822, 1296, 642, 0, 0, - 1979, 937, 0, 0, 0, 0, 0, 1980, 1981, 0, - 3000, 1982, 1983, 1984, 111, 112, 113, 114, 115, 116, - 117, 118, 1128, 119, 120, 121, 1129, 1130, 1131, 2823, - 1132, 1133, 1134, 1135, 2824, 123, 124, 1136, 125, 126, - 127, 2825, 129, 130, 131, 0, 1435, 2826, 1437, 1438, - 1137, 137, 138, 139, 140, 141, 142, 1138, 1139, 143, - 144, 145, 146, 1439, 1440, 149, 1140, 150, 151, 152, - 153, 0, 1141, 2827, 1142, 2828, 157, 158, 159, 160, - 161, 2829, 163, 164, 165, 1143, 166, 167, 168, 169, - 170, 171, 1144, 2830, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 1445, 184, 185, 1446, 187, 1145, - 188, 1146, 189, 190, 191, 192, 193, 194, 1147, 195, - 196, 197, 198, 199, 1148, 1149, 200, 201, 950, 203, - 204, 1150, 205, 206, 207, 1151, 208, 209, 210, 1152, - 211, 212, 213, 214, 0, 216, 217, 218, 219, 220, - 0, 1153, 222, 1154, 223, 224, 1447, 226, 1155, 227, - 1156, 228, 2831, 1157, 2832, 231, 232, 2833, 2834, 235, - 1158, 236, 1159, 0, 0, 239, 240, 1160, 241, 242, - 243, 244, 245, 246, 247, 2835, 249, 250, 251, 252, - 1161, 253, 254, 255, 256, 257, 258, 259, 1162, 260, - 2836, 0, 263, 264, 265, 266, 267, 1453, 1454, 1163, - 1455, 1164, 271, 2837, 2838, 274, 2839, 276, 277, 278, - 279, 280, 281, 1165, 1166, 282, 2840, 284, 2841, 1167, - 286, 287, 288, 289, 290, 291, 292, 293, 2842, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 1462, - 2843, 1464, 318, 319, 320, 2844, 1168, 322, 323, 2845, - 325, 1169, 0, 327, 1466, 329, 330, 331, 1170, 332, - 333, 1171, 1172, 2846, 335, 336, 1173, 1174, 337, 338, - 0, 2847, 341, 2848, 0, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 1175, 1176, 1177, 1178, 354, - 355, 0, 2849, 358, 359, 0, 361, 362, 363, 1179, - 364, 365, 366, 367, 368, 369, 1180, 370, 371, 372, - 1470, 374, 375, 376, 377, 1181, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 1182, - 391, 392, 2850, 394, 395, 396, 1472, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 1183, 2851, - 409, 410, 411, 412, 413, 414, 2852, 416, 417, 418, - 2853, 420, 421, 1476, 423, 1184, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 2854, 437, - 0, 439, 1185, 440, 441, 1186, 442, 2855, 444, 445, - 446, 447, 448, 1187, 449, 1479, 1480, 1188, 1189, 452, - 453, 0, 455, 0, 1190, 457, 458, 2856, 460, 461, - 462, 463, 464, 2857, 1192, 465, 466, 467, 1193, 468, - 469, 470, 471, 1194, 472, 473, 474, 475, 476, 0, - 1483, 1196, 479, 2858, 481, 482, 483, 484, 485, 486, - 487, 1197, 1198, 488, 1199, 1200, 489, 490, 491, 492, - 493, 494, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 506, 507, 508, 509, 0, 517, 0, - 1985, 1986, 1987, 1979, 2859, 2860, 1990, 1991, 1992, 1993, - 1980, 1981, 0, 0, 1982, 1983, 1984, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 0, 0, 0, 1985, 1986, 1987, 0, 1988, 1989, 1990, - 1991, 1992, 1993, 1566, 0, 0, 1567, 0, 0, 0, - 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, + 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, + 509, 510, 511, 512, 513, 514, 515, 0, 2885, 1339, + 815, 0, 0, 2028, 1046, 0, 0, 0, 0, 0, + 2029, 2030, 0, 3067, 2031, 2032, 2033, 117, 118, 119, + 120, 121, 122, 123, 124, 556, 125, 126, 127, 557, + 558, 559, 2886, 561, 562, 563, 564, 2887, 129, 130, + 566, 131, 132, 133, 2888, 135, 136, 137, 0, 1476, + 2889, 1478, 1479, 573, 143, 144, 145, 146, 147, 148, + 574, 575, 149, 150, 151, 152, 1480, 1481, 155, 578, + 156, 157, 158, 159, 0, 580, 2890, 582, 2891, 163, + 164, 165, 166, 167, 2892, 169, 170, 171, 585, 172, + 173, 174, 175, 176, 177, 586, 2893, 179, 180, 181, + 182, 183, 184, 185, 186, 187, 188, 1486, 190, 191, + 1487, 193, 591, 194, 592, 195, 196, 197, 198, 199, + 200, 593, 594, 201, 202, 203, 204, 595, 596, 205, + 206, 1059, 208, 209, 597, 210, 211, 212, 598, 213, + 214, 215, 599, 216, 217, 218, 219, 0, 221, 222, + 223, 224, 225, 0, 602, 227, 603, 228, 229, 1488, + 231, 605, 232, 606, 233, 2894, 608, 2895, 236, 237, + 2896, 2897, 240, 612, 241, 613, 0, 0, 244, 245, + 616, 246, 247, 248, 249, 250, 251, 252, 2898, 254, + 255, 256, 257, 618, 258, 259, 260, 261, 262, 263, + 264, 619, 265, 2899, 0, 268, 269, 270, 271, 272, + 1494, 1495, 624, 1496, 626, 276, 2900, 2901, 279, 2902, + 281, 282, 283, 284, 285, 286, 630, 631, 287, 2903, + 289, 2904, 634, 291, 292, 293, 294, 295, 296, 297, + 298, 2905, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, + 318, 319, 1503, 2906, 1505, 323, 324, 325, 2907, 640, + 327, 328, 2908, 330, 642, 0, 332, 1507, 334, 335, + 336, 645, 337, 338, 646, 647, 2909, 340, 341, 648, + 649, 342, 343, 0, 2910, 346, 2911, 0, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 358, 359, 654, + 655, 656, 657, 360, 361, 0, 2912, 364, 365, 0, + 367, 368, 369, 661, 370, 371, 372, 373, 374, 375, + 662, 376, 377, 378, 1511, 380, 381, 382, 383, 664, + 384, 385, 386, 387, 388, 389, 390, 391, 392, 393, + 394, 395, 396, 665, 397, 398, 2913, 400, 401, 402, + 1513, 404, 405, 406, 407, 408, 409, 410, 411, 412, + 413, 414, 415, 416, 668, 2914, 417, 418, 419, 420, + 421, 422, 2915, 424, 425, 671, 2916, 427, 428, 1517, + 430, 674, 431, 432, 433, 434, 435, 436, 437, 438, + 439, 440, 441, 442, 2917, 444, 0, 677, 678, 446, + 447, 679, 448, 2918, 450, 451, 452, 453, 454, 681, + 455, 1520, 1521, 684, 685, 458, 459, 0, 461, 0, + 688, 463, 464, 2919, 466, 467, 468, 469, 470, 2920, + 691, 471, 472, 473, 692, 474, 475, 476, 477, 693, + 478, 479, 480, 481, 482, 0, 1524, 696, 485, 2921, + 487, 488, 489, 490, 491, 492, 493, 698, 699, 494, + 700, 701, 495, 496, 497, 498, 499, 500, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 512, + 513, 514, 515, 0, 523, 0, 2034, 2035, 2036, 2028, + 2922, 2923, 2039, 2040, 2041, 2042, 2029, 2030, 0, 0, + 2031, 2032, 2033, 117, 118, 119, 120, 121, 122, 123, + 124, 0, 125, 126, 127, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 129, 130, 0, 131, 132, 133, + 0, 135, 136, 137, 138, 139, 0, 141, 142, 0, + 143, 144, 145, 146, 147, 148, 0, 0, 149, 150, + 151, 152, 153, 154, 155, 0, 156, 157, 158, 159, + 160, 0, 0, 0, 162, 163, 164, 165, 166, 167, + 0, 169, 170, 171, 0, 172, 173, 174, 175, 176, + 177, 0, 0, 179, 180, 181, 182, 183, 184, 185, + 186, 187, 188, 189, 190, 191, 192, 193, 0, 194, + 0, 195, 196, 197, 198, 199, 200, 0, 0, 201, + 202, 203, 204, 0, 0, 205, 206, 207, 208, 209, + 0, 210, 211, 212, 0, 213, 214, 215, 0, 216, + 217, 218, 219, 220, 221, 222, 223, 224, 225, 226, + 0, 227, 0, 228, 229, 230, 231, 0, 232, 0, + 233, 0, 0, 0, 236, 237, 524, 0, 240, 0, + 241, 0, 242, 243, 244, 245, 0, 246, 247, 248, + 249, 250, 251, 252, 0, 254, 255, 256, 257, 0, + 258, 259, 260, 261, 262, 263, 264, 0, 265, 0, + 267, 268, 269, 270, 271, 272, 273, 274, 0, 275, + 0, 276, 0, 0, 279, 0, 281, 282, 283, 284, + 285, 286, 0, 0, 287, 0, 289, 0, 0, 291, + 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, 314, 315, 316, 317, 318, 319, 320, 0, + 322, 323, 324, 325, 326, 0, 327, 328, 0, 330, + 0, 331, 332, 333, 334, 335, 336, 0, 337, 338, + 0, 0, 339, 340, 341, 0, 0, 342, 343, 344, + 0, 346, 0, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, 358, 359, 0, 0, 0, 0, 360, + 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, + 370, 371, 372, 373, 374, 375, 0, 376, 377, 378, + 379, 380, 381, 382, 383, 0, 384, 385, 386, 387, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 0, + 397, 398, 0, 400, 401, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 412, 413, 414, 415, 416, + 0, 0, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 0, 0, 427, 428, 429, 430, 0, 431, 432, + 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, + 526, 444, 445, 0, 0, 446, 447, 0, 448, 0, + 450, 451, 452, 453, 454, 0, 455, 456, 457, 0, + 0, 458, 459, 460, 461, 462, 0, 463, 464, 465, + 466, 467, 468, 469, 470, 0, 0, 471, 472, 473, + 0, 474, 475, 476, 477, 0, 478, 479, 480, 481, + 482, 483, 484, 0, 485, 0, 487, 488, 489, 490, + 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, + 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, + 507, 508, 509, 510, 511, 512, 513, 514, 515, 0, + 0, 0, 2034, 2035, 2036, 0, 2037, 2038, 2039, 2040, + 2041, 2042, 1607, 0, 0, 1608, 0, 0, 0, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1575, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1577, 1566, 0, 0, 1567, 0, 0, 1578, 1568, - 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, + 1616, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1618, 1607, 0, 0, 1608, 0, 0, 1619, 1609, 1610, + 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1616, + 0, 0, 0, 0, 1620, 0, 0, 0, 0, 1618, + 1607, 0, 0, 1608, 0, 0, 1619, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1616, 0, + 0, 0, 0, 1620, 0, 0, 0, 0, 1618, 0, + 0, 0, 0, 0, 0, 1619, 0, 1607, 0, 0, + 1608, 0, 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, + 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1620, 0, 0, 1616, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1618, 1607, 0, 0, 1608, + 1621, 0, 1619, 1609, 1610, 1611, 1612, 1613, 1614, 1615, + 0, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 0, 0, 1623, 0, 1616, 0, 0, 0, 0, 1620, + 0, 0, 0, 0, 1618, 0, 0, 0, 0, 1621, + 0, 1619, 0, 0, 0, 1624, 1625, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1622, 0, 0, 0, + 1626, 1623, 0, 0, 0, 0, 0, 0, 1620, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1621, 0, + 0, 0, 0, 0, 1624, 1625, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1622, 0, 0, 1627, 1626, + 1623, 1628, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1629, 0, 0, 1630, 0, + 0, 0, 0, 1624, 1625, 1621, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1627, 1626, 0, + 1628, 0, 1622, 0, 0, 0, 0, 1623, 0, 0, + 0, 0, 0, 0, 1629, 0, 0, 1630, 0, 0, + 0, 0, 0, 0, 1621, 0, 0, 0, 0, 0, + 1624, 1625, 0, 0, 0, 0, 1627, 0, 0, 1628, + 0, 1622, 0, 0, 0, 1626, 1623, 0, 0, 0, + 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1624, + 1625, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 0, 0, 1627, 1626, 0, 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1575, 0, 0, 0, 0, 1579, 0, 0, 0, 0, - 1577, 1566, 0, 0, 1567, 0, 0, 1578, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1575, - 0, 0, 0, 0, 1579, 0, 0, 0, 0, 1577, - 0, 0, 0, 0, 0, 0, 1578, 0, 1566, 0, - 0, 1567, 0, 0, 0, 1568, 1569, 1570, 1571, 1572, - 1573, 1574, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1579, 0, 0, 1575, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1577, 1566, 0, 0, - 1567, 1580, 0, 1578, 1568, 1569, 1570, 1571, 1572, 1573, - 1574, 0, 0, 0, 0, 0, 0, 0, 1581, 0, - 0, 0, 0, 1582, 0, 1575, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 1577, 0, 0, 0, 0, - 1580, 0, 1578, 0, 0, 0, 1583, 1584, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1581, 0, 0, - 0, 1585, 1582, 0, 0, 0, 0, 0, 0, 1579, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, - 0, 0, 0, 0, 0, 1583, 1584, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1581, 0, 0, 1586, - 1585, 1582, 1587, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1588, 0, 0, 1589, - 0, 0, 0, 0, 1583, 1584, 1580, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1586, 1585, - 0, 1587, 0, 1581, 0, 0, 0, 0, 1582, 0, - 0, 0, 0, 0, 0, 1588, 0, 0, 1589, 0, - 0, 0, 0, 0, 0, 1580, 0, 0, 0, 0, - 0, 1583, 1584, 0, 0, 0, 0, 1586, 0, 0, - 1587, 0, 1581, 0, 0, 0, 1585, 1582, 0, 0, - 0, 0, 0, 0, 1588, 0, 0, 1589, 0, 0, + 1629, 0, 0, 1630, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1631, 0, 0, 0, 0, 0, + 0, 0, 1627, 0, 0, 1628, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1629, + 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1583, 1584, 0, 1590, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1586, 1585, 0, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1588, 0, 0, 1589, 0, 0, 0, 0, 0, - 0, 0, 1590, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1586, 0, 0, 1587, 0, 0, 0, + 0, 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 0, 0, 0, 0, 2692, + 1631, 0, 0, 0, 0, 0, 1607, 0, 0, 1608, + 0, 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, + 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, + 1638, 1639, 1640, 1641, 1616, 0, 0, 0, 2764, 1631, + 0, 0, 0, 0, 1618, 0, 0, 0, 0, 0, + 0, 1619, 0, 0, 0, 0, 0, 0, 0, 0, + 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 0, 0, 0, 0, 2831, 1620, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1588, 0, 0, 1589, 0, 0, 0, 0, 0, 0, - 0, 1590, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1607, 0, 0, 1608, 0, 0, 0, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 1632, 0, 0, + 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, + 1616, 0, 0, 0, 3059, 0, 0, 0, 0, 0, + 1618, 1607, 0, 0, 1608, 0, 0, 1619, 1609, 1610, + 1611, 1612, 1613, 1614, 1615, 0, 1632, 0, 0, 1633, + 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 1616, + 0, 0, 0, 3066, 1620, 0, 0, 0, 0, 1618, + 1607, 0, 0, 1608, 1621, 0, 1619, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, + 0, 1622, 0, 0, 0, 0, 1623, 0, 1616, 0, + 0, 0, 0, 1620, 0, 0, 0, 0, 1618, 0, + 0, 0, 0, 0, 0, 1619, 0, 0, 0, 1624, + 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1626, 0, 0, 0, 0, 0, + 0, 0, 1620, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1607, 0, 0, 1608, + 1621, 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, + 0, 0, 1627, 0, 0, 1628, 0, 1622, 0, 0, + 0, 0, 1623, 0, 1616, 0, 0, 0, 0, 1629, + 0, 0, 1630, 0, 1618, 0, 0, 0, 0, 1621, + 0, 1619, 0, 0, 0, 1624, 1625, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1622, 0, 0, 0, + 1626, 1623, 0, 0, 0, 0, 0, 0, 1620, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1621, 0, + 0, 0, 0, 0, 1624, 1625, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1622, 0, 0, 1627, 1626, + 1623, 1628, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1629, 0, 0, 1630, 0, + 0, 0, 0, 1624, 1625, 0, 0, 0, 0, 1631, + 0, 0, 0, 0, 0, 0, 0, 1627, 1626, 0, + 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1629, 0, 0, 1630, 0, 0, + 0, 0, 0, 0, 1621, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1627, 0, 0, 1628, + 0, 1622, 0, 0, 0, 0, 1623, 0, 0, 0, + 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1624, + 1625, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 0, 0, 0, 1626, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1632, 0, 0, 1633, + 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, + 0, 0, 0, 3227, 1631, 0, 0, 0, 0, 0, + 0, 0, 1627, 0, 1607, 1628, 0, 1608, 0, 0, + 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 1629, + 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1616, 1631, 0, 0, 0, 0, 0, 0, + 0, 0, 1618, 0, 0, 0, 0, 0, 0, 1619, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 0, 1620, 0, 0, 3249, + 0, 0, 0, 0, 0, 0, 1607, 0, 0, 1608, + 0, 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, + 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, + 1638, 1639, 1640, 1641, 1616, 0, 0, 0, 3351, 1631, + 0, 0, 0, 0, 1618, 1607, 0, 0, 1608, 0, + 0, 1619, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, + 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 1616, 0, 0, 0, 3409, 1620, 0, + 0, 0, 0, 1618, 0, 0, 0, 0, 0, 0, + 1619, 0, 1621, 0, 0, 1607, 0, 0, 1608, 0, + 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1622, + 0, 0, 0, 0, 1623, 0, 0, 1620, 0, 0, + 0, 0, 0, 1616, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1618, 0, 0, 0, 1624, 1625, 0, + 1619, 0, 0, 0, 0, 0, 1632, 0, 0, 1633, + 1634, 1635, 1626, 1636, 1637, 1638, 1639, 1640, 1641, 0, + 0, 0, 0, 3431, 0, 0, 0, 1620, 0, 0, + 0, 0, 0, 0, 1621, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, - 1598, 1599, 1600, 0, 0, 0, 0, 2428, 1590, 0, - 0, 0, 0, 0, 1566, 0, 0, 1567, 0, 0, - 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 1591, - 0, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, - 1599, 1600, 1575, 0, 0, 0, 2629, 1590, 0, 0, - 0, 0, 1577, 0, 0, 0, 0, 0, 0, 1578, - 0, 0, 0, 0, 0, 0, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 0, 0, 0, 0, 2701, 1579, 0, 0, 0, + 1627, 1622, 0, 1628, 0, 0, 1623, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1629, 0, 0, + 1630, 0, 0, 1621, 0, 0, 0, 0, 0, 1624, + 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1622, 0, 0, 0, 1626, 1623, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, 1570, - 1571, 1572, 1573, 1574, 0, 1591, 0, 0, 1592, 1593, - 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 1575, 0, - 0, 0, 2768, 0, 0, 0, 0, 0, 1577, 1566, - 0, 0, 1567, 0, 0, 1578, 1568, 1569, 1570, 1571, - 1572, 1573, 1574, 0, 1591, 0, 0, 1592, 1593, 1594, - 0, 1595, 1596, 1597, 1598, 1599, 1600, 1575, 0, 0, - 0, 2992, 1579, 0, 0, 0, 0, 1577, 1566, 0, - 0, 1567, 1580, 0, 1578, 1568, 1569, 1570, 1571, 1572, - 1573, 1574, 0, 0, 0, 0, 0, 0, 0, 1581, - 0, 0, 0, 0, 1582, 0, 1575, 0, 0, 0, - 0, 1579, 0, 0, 0, 0, 1577, 0, 0, 0, - 0, 0, 0, 1578, 0, 0, 0, 1583, 1584, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1624, 1625, + 0, 0, 0, 1621, 0, 0, 0, 0, 0, 0, + 0, 0, 1627, 1626, 0, 1628, 0, 0, 0, 0, + 1622, 0, 0, 0, 0, 1623, 0, 0, 0, 1629, + 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1631, 1624, 1625, + 0, 1627, 0, 0, 1628, 0, 0, 0, 0, 0, + 0, 0, 0, 1626, 0, 0, 0, 0, 1629, 0, + 0, 1630, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1585, 0, 0, 0, 0, 0, 0, 0, - 1579, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1566, 0, 0, 1567, 1580, 0, - 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, - 1586, 0, 0, 1587, 0, 1581, 0, 0, 0, 0, - 1582, 0, 1575, 0, 0, 0, 0, 1588, 0, 0, - 1589, 0, 1577, 0, 0, 0, 0, 1580, 0, 1578, - 0, 0, 0, 1583, 1584, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1581, 0, 0, 0, 1585, 1582, - 0, 0, 0, 0, 0, 0, 1579, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1580, 0, 0, 0, - 0, 0, 1583, 1584, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1581, 0, 0, 1586, 1585, 1582, 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1588, 0, 0, 1589, 0, 0, 0, - 0, 1583, 1584, 0, 1590, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1586, 1585, 0, 1587, 0, + 0, 1627, 0, 0, 1628, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1629, 0, + 0, 1630, 0, 0, 0, 0, 0, 0, 0, 1631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1588, 0, 0, 1589, 0, 0, 0, 0, - 0, 0, 1580, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1586, 0, 0, 1587, 0, 1581, - 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, - 0, 1588, 0, 0, 1589, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1583, 1584, 0, - 1590, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1585, 0, 0, 0, 0, 0, 0, 0, - 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, - 1597, 1598, 1599, 1600, 0, 0, 0, 0, 2999, 1590, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1566, - 1586, 0, 1567, 1587, 0, 0, 1568, 1569, 1570, 1571, - 1572, 1573, 1574, 0, 0, 0, 0, 1588, 0, 0, - 1589, 0, 0, 0, 0, 0, 0, 1575, 1590, 0, - 0, 0, 0, 0, 0, 0, 0, 1577, 0, 0, - 0, 0, 0, 0, 1578, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1591, 0, 0, - 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, - 0, 1579, 0, 0, 3157, 0, 0, 0, 0, 0, - 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 0, 1591, 0, 0, 1592, - 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 1575, - 0, 0, 0, 3179, 1590, 0, 0, 0, 0, 1577, - 1566, 0, 0, 1567, 0, 0, 1578, 1568, 1569, 1570, - 1571, 1572, 1573, 1574, 0, 1591, 0, 0, 1592, 1593, - 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 1575, 0, - 0, 0, 3279, 1579, 0, 0, 0, 0, 1577, 0, - 0, 0, 0, 0, 0, 1578, 0, 1580, 0, 1566, - 0, 0, 1567, 0, 0, 0, 1568, 1569, 1570, 1571, - 1572, 1573, 1574, 0, 1581, 0, 0, 0, 0, 1582, - 0, 0, 1579, 0, 0, 0, 0, 1575, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1577, 0, 0, - 0, 0, 1583, 1584, 1578, 0, 0, 0, 0, 0, - 0, 1591, 0, 0, 1592, 1593, 1594, 1585, 1595, 1596, - 1597, 1598, 1599, 1600, 0, 0, 0, 0, 3336, 0, - 0, 1579, 0, 0, 0, 0, 0, 0, 0, 1580, - 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 1586, 1581, 0, 1587, 0, - 0, 1582, 0, 0, 0, 0, 0, 0, 0, 1575, - 0, 0, 1588, 0, 0, 1589, 0, 0, 1580, 1577, - 0, 0, 0, 0, 1583, 1584, 1578, 0, 0, 0, - 0, 0, 0, 0, 0, 1581, 0, 0, 0, 1585, - 1582, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1579, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1583, 1584, 0, 0, 1580, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1586, 1585, 0, - 1587, 0, 0, 0, 1581, 0, 0, 0, 0, 1582, - 0, 0, 0, 0, 1588, 0, 0, 1589, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1590, - 0, 0, 1583, 1584, 0, 0, 1586, 0, 0, 1587, - 0, 0, 0, 0, 0, 0, 0, 1585, 0, 0, - 0, 0, 0, 1588, 0, 0, 1589, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1580, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1586, 1581, 0, 1587, 0, - 0, 1582, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1588, 0, 0, 1589, 0, 0, 0, 0, - 0, 1590, 0, 0, 1583, 1584, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 0, 0, 0, 0, 1591, 0, 0, 1592, - 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 0, - 1590, 0, 0, 3358, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1586, 0, 0, - 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1588, 0, 0, 1589, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1590, + 0, 0, 0, 0, 1632, 0, 0, 1633, 1634, 1635, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 1631, 1792, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 0, 0, 1751, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1591, 0, 0, - 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, - 0, 0, 2722, 0, 0, 0, 0, 0, 0, 0, - 0, 1590, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1591, 0, 0, 1592, - 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, 0, - 0, 3147, 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, 1631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1632, 0, 0, 1633, + 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, + 0, 2785, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1632, 0, 0, 1633, 1634, + 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, + 3217, 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, 1127, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 0, 0, 3320, 111, 112, 113, 114, 115, 116, - 117, 118, 1128, 119, 120, 121, 1129, 1130, 1131, 776, - 1132, 1133, 1134, 1135, 777, 123, 124, 1136, 125, 126, - 127, 778, 129, 130, 131, 779, 780, 781, 782, 783, - 1137, 137, 138, 139, 140, 141, 142, 1138, 1139, 143, - 144, 145, 146, 784, 785, 149, 1140, 150, 151, 152, - 153, 786, 1141, 787, 1142, 788, 157, 158, 159, 160, - 161, 789, 163, 164, 165, 1143, 166, 167, 168, 169, - 170, 171, 1144, 790, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 791, 184, 185, 792, 187, 1145, - 188, 1146, 189, 190, 191, 192, 193, 194, 1147, 195, - 196, 197, 198, 199, 1148, 1149, 200, 201, 202, 203, - 204, 1150, 205, 206, 207, 1151, 208, 209, 210, 1152, - 211, 212, 213, 214, 793, 216, 217, 218, 219, 220, - 794, 1153, 222, 1154, 223, 224, 795, 226, 1155, 227, - 1156, 228, 796, 1157, 797, 231, 232, 798, 799, 235, - 1158, 236, 1159, 800, 801, 239, 240, 1160, 241, 242, - 243, 244, 245, 246, 247, 802, 249, 250, 251, 252, - 1161, 253, 254, 255, 256, 257, 258, 259, 1162, 260, - 803, 804, 263, 264, 265, 266, 267, 805, 806, 1163, - 807, 1164, 271, 808, 809, 274, 810, 276, 277, 278, - 279, 280, 281, 1165, 1166, 282, 811, 284, 812, 1167, - 286, 287, 288, 289, 290, 291, 292, 293, 813, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 814, - 815, 816, 318, 319, 320, 817, 1168, 322, 323, 818, - 325, 1169, 819, 327, 820, 329, 330, 331, 1170, 332, - 333, 1171, 1172, 334, 335, 336, 1173, 1174, 337, 338, - 821, 822, 341, 823, 824, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 1175, 1176, 1177, 1178, 354, - 355, 825, 826, 358, 359, 827, 361, 362, 363, 1179, - 364, 365, 366, 367, 368, 369, 1180, 370, 371, 372, - 828, 374, 375, 376, 377, 1181, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 1182, - 391, 392, 829, 394, 395, 396, 830, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 1183, 831, - 409, 410, 411, 412, 413, 414, 832, 416, 417, 418, - 833, 420, 421, 834, 423, 1184, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 835, 437, - 836, 439, 1185, 440, 441, 1186, 442, 837, 444, 445, - 446, 447, 448, 1187, 449, 838, 839, 1188, 1189, 452, - 453, 840, 455, 841, 1190, 457, 458, 842, 460, 461, - 462, 463, 464, 1191, 1192, 465, 466, 467, 1193, 468, - 469, 470, 471, 1194, 472, 473, 474, 475, 476, 1195, - 844, 1196, 479, 845, 481, 482, 483, 484, 485, 486, - 487, 1197, 1198, 488, 1199, 1200, 489, 490, 491, 492, - 493, 494, 846, 847, 848, 849, 850, 851, 852, 853, - 854, 855, 856, 506, 507, 508, 509, 517, 0, 0, - 0, 0, 0, 0, 0, 0, 2014, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 123, 124, 0, - 125, 126, 127, 0, 129, 130, 131, 132, 133, 0, - 135, 136, 0, 137, 138, 139, 140, 141, 142, 0, - 0, 143, 144, 145, 146, 147, 148, 149, 0, 150, - 151, 152, 153, 154, 0, 0, 0, 156, 157, 158, - 159, 160, 161, 0, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 0, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 202, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 221, 0, 222, 0, 223, 224, 225, 226, - 0, 227, 0, 228, 0, 0, 0, 231, 232, 518, - 0, 235, 0, 236, 0, 237, 238, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 0, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 0, 262, 263, 264, 265, 266, 267, 268, - 269, 0, 270, 0, 271, 0, 0, 274, 0, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 0, 284, - 0, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 519, 295, 296, 297, 298, 299, 300, 301, 302, 303, + 0, 0, 555, 0, 0, 1632, 0, 0, 1633, 1634, + 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, + 3393, 117, 118, 119, 120, 121, 122, 123, 124, 556, + 125, 126, 127, 557, 558, 559, 560, 561, 562, 563, + 564, 565, 129, 130, 566, 131, 132, 133, 567, 135, + 136, 137, 568, 569, 570, 571, 572, 573, 143, 144, + 145, 146, 147, 148, 574, 575, 149, 150, 151, 152, + 576, 577, 155, 578, 156, 157, 158, 159, 579, 580, + 581, 582, 583, 163, 164, 165, 166, 167, 584, 169, + 170, 171, 585, 172, 173, 174, 175, 176, 177, 586, + 587, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 589, 190, 191, 590, 193, 591, 194, 592, 195, + 196, 197, 198, 199, 200, 593, 594, 201, 202, 203, + 204, 595, 596, 205, 206, 207, 208, 209, 597, 210, + 211, 212, 598, 213, 214, 215, 599, 216, 217, 218, + 219, 600, 221, 222, 223, 224, 225, 601, 602, 227, + 603, 228, 229, 604, 231, 605, 232, 606, 233, 607, + 608, 609, 236, 237, 610, 611, 240, 612, 241, 613, + 614, 615, 244, 245, 616, 246, 247, 248, 249, 250, + 251, 252, 617, 254, 255, 256, 257, 618, 258, 259, + 260, 261, 262, 263, 264, 619, 265, 620, 621, 268, + 269, 270, 271, 272, 622, 623, 624, 625, 626, 276, + 627, 628, 279, 629, 281, 282, 283, 284, 285, 286, + 630, 631, 287, 632, 289, 633, 634, 291, 292, 293, + 294, 295, 296, 297, 298, 635, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 0, 317, 318, 319, 320, 321, 0, 322, - 323, 0, 325, 0, 326, 327, 328, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 0, 341, 0, 343, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 356, 0, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 373, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 0, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 0, 420, 421, 422, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 520, 437, 438, 439, 0, 440, 441, 0, 442, 0, - 444, 445, 446, 447, 448, 0, 449, 450, 451, 0, - 0, 452, 453, 454, 455, 456, 0, 457, 458, 459, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 478, 0, 479, 0, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 517, - 0, 0, 0, 0, 0, 0, 0, 0, 2622, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 123, - 124, 0, 125, 126, 127, 0, 129, 130, 131, 132, - 133, 0, 135, 136, 0, 137, 138, 139, 140, 141, - 142, 0, 0, 143, 144, 145, 146, 147, 148, 149, - 0, 150, 151, 152, 153, 154, 0, 0, 0, 156, - 157, 158, 159, 160, 161, 0, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 0, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 202, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 215, 216, - 217, 218, 219, 220, 221, 0, 222, 0, 223, 224, - 225, 226, 0, 227, 0, 228, 0, 0, 0, 231, - 232, 518, 0, 235, 0, 236, 0, 237, 238, 239, - 240, 0, 241, 242, 243, 244, 245, 246, 247, 0, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 0, 262, 263, 264, 265, 266, - 267, 268, 269, 0, 270, 0, 271, 0, 0, 274, - 0, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 0, 284, 0, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 519, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 315, 0, 317, 318, 319, 320, 321, - 0, 322, 323, 0, 325, 0, 326, 327, 328, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 0, 341, 0, 343, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 356, 0, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 373, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 0, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 0, 420, 421, 422, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 520, 437, 438, 439, 0, 440, 441, 0, - 442, 0, 444, 445, 446, 447, 448, 0, 449, 450, - 451, 0, 0, 452, 453, 454, 455, 456, 0, 457, - 458, 459, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 478, 0, 479, 0, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, + 314, 315, 316, 317, 318, 319, 636, 637, 638, 323, + 324, 325, 639, 640, 327, 328, 641, 330, 642, 643, + 332, 644, 334, 335, 336, 645, 337, 338, 646, 647, + 339, 340, 341, 648, 649, 342, 343, 650, 651, 346, + 652, 653, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 654, 655, 656, 657, 360, 361, 658, + 659, 364, 365, 660, 367, 368, 369, 661, 370, 371, + 372, 373, 374, 375, 662, 376, 377, 378, 663, 380, + 381, 382, 383, 664, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 665, 397, 398, + 666, 400, 401, 402, 667, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 668, 669, + 417, 418, 419, 420, 421, 422, 670, 424, 425, 671, + 672, 427, 428, 673, 430, 674, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 675, 444, + 676, 677, 678, 446, 447, 679, 448, 680, 450, 451, + 452, 453, 454, 681, 455, 682, 683, 684, 685, 458, + 459, 686, 461, 687, 688, 463, 464, 689, 466, 467, + 468, 469, 470, 690, 691, 471, 472, 473, 692, 474, + 475, 476, 477, 693, 478, 479, 480, 481, 482, 694, + 695, 696, 485, 697, 487, 488, 489, 490, 491, 492, + 493, 698, 699, 494, 700, 701, 495, 496, 497, 498, + 499, 500, 702, 703, 704, 705, 706, 707, 708, 709, + 710, 711, 712, 512, 513, 514, 515, 523, 0, 0, + 0, 0, 0, 0, 0, 0, 2063, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 129, 130, 0, + 131, 132, 133, 0, 135, 136, 137, 138, 139, 0, + 141, 142, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 153, 154, 155, 0, 156, + 157, 158, 159, 160, 0, 0, 0, 162, 163, 164, + 165, 166, 167, 0, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 0, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 189, 190, 191, 192, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 220, 221, 222, 223, + 224, 225, 226, 0, 227, 0, 228, 229, 230, 231, + 0, 232, 0, 233, 0, 0, 0, 236, 237, 524, + 0, 240, 0, 241, 0, 242, 243, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 0, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 0, 267, 268, 269, 270, 271, 272, 273, + 274, 0, 275, 0, 276, 0, 0, 279, 0, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 0, 289, + 0, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 320, 0, 322, 323, 324, 325, 326, 0, 327, + 328, 0, 330, 0, 331, 332, 333, 334, 335, 336, + 0, 337, 338, 0, 0, 339, 340, 341, 0, 0, + 342, 343, 344, 0, 346, 0, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 379, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 0, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 0, 417, 418, 419, 420, 421, + 422, 423, 424, 425, 0, 0, 427, 428, 429, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 526, 444, 445, 0, 0, 446, 447, + 0, 448, 0, 450, 451, 452, 453, 454, 0, 455, + 456, 457, 0, 0, 458, 459, 460, 461, 462, 0, + 463, 464, 465, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 483, 484, 0, 485, 0, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, + 504, 505, 506, 507, 508, 509, 510, 511, 512, 513, + 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, + 0, 2685, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 117, 118, 119, 120, 121, 122, 123, 124, 0, + 125, 126, 127, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 129, 130, 0, 131, 132, 133, 0, 135, + 136, 137, 138, 139, 0, 141, 142, 0, 143, 144, + 145, 146, 147, 148, 0, 0, 149, 150, 151, 152, + 153, 154, 155, 0, 156, 157, 158, 159, 160, 0, + 0, 0, 162, 163, 164, 165, 166, 167, 0, 169, + 170, 171, 0, 172, 173, 174, 175, 176, 177, 0, + 0, 179, 180, 181, 182, 183, 184, 185, 186, 187, + 188, 189, 190, 191, 192, 193, 0, 194, 0, 195, + 196, 197, 198, 199, 200, 0, 0, 201, 202, 203, + 204, 0, 0, 205, 206, 207, 208, 209, 0, 210, + 211, 212, 0, 213, 214, 215, 0, 216, 217, 218, + 219, 220, 221, 222, 223, 224, 225, 226, 0, 227, + 0, 228, 229, 230, 231, 0, 232, 0, 233, 0, + 0, 0, 236, 237, 524, 0, 240, 0, 241, 0, + 242, 243, 244, 245, 0, 246, 247, 248, 249, 250, + 251, 252, 0, 254, 255, 256, 257, 0, 258, 259, + 260, 261, 262, 263, 264, 0, 265, 0, 267, 268, + 269, 270, 271, 272, 273, 274, 0, 275, 0, 276, + 0, 0, 279, 0, 281, 282, 283, 284, 285, 286, + 0, 0, 287, 0, 289, 0, 0, 291, 292, 293, + 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 314, 315, 316, 317, 318, 319, 320, 0, 322, 323, + 324, 325, 326, 0, 327, 328, 0, 330, 0, 331, + 332, 333, 334, 335, 336, 0, 337, 338, 0, 0, + 339, 340, 341, 0, 0, 342, 343, 344, 0, 346, + 0, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, 358, 359, 0, 0, 0, 0, 360, 361, 362, + 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, + 372, 373, 374, 375, 0, 376, 377, 378, 379, 380, + 381, 382, 383, 0, 384, 385, 386, 387, 388, 389, + 390, 391, 392, 393, 394, 395, 396, 0, 397, 398, + 0, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, 412, 413, 414, 415, 416, 0, 0, + 417, 418, 419, 420, 421, 422, 423, 424, 425, 0, + 0, 427, 428, 429, 430, 0, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 526, 444, + 445, 0, 0, 446, 447, 0, 448, 0, 450, 451, + 452, 453, 454, 0, 455, 456, 457, 0, 0, 458, + 459, 460, 461, 462, 0, 463, 464, 465, 466, 467, + 468, 469, 470, 0, 0, 471, 472, 473, 0, 474, + 475, 476, 477, 0, 478, 479, 480, 481, 482, 483, + 484, 0, 485, 0, 487, 488, 489, 490, 491, 492, + 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 775, 1296, 642, 0, 0, 0, 937, 0, 0, - 2625, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 776, 0, 0, 0, 0, - 777, 123, 124, 0, 125, 126, 127, 778, 129, 130, - 131, 779, 780, 781, 782, 783, 0, 137, 138, 139, - 140, 141, 142, 0, 0, 143, 144, 145, 146, 784, - 785, 149, 0, 150, 151, 152, 153, 786, 0, 787, - 0, 788, 157, 158, 159, 160, 161, 789, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 790, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 791, 184, 185, 792, 187, 0, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 202, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 793, 216, 217, 218, 219, 220, 794, 1297, 222, 0, - 223, 224, 795, 226, 0, 227, 0, 228, 796, 0, - 797, 231, 232, 798, 799, 235, 0, 236, 0, 800, - 801, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 802, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 803, 804, 263, 264, - 265, 266, 267, 805, 806, 0, 807, 0, 271, 808, - 809, 274, 810, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 811, 284, 812, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 813, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 814, 815, 816, 318, 319, - 320, 817, 0, 322, 323, 818, 325, 0, 819, 327, - 820, 329, 330, 331, 0, 332, 333, 1298, 0, 334, - 335, 336, 0, 0, 337, 338, 821, 822, 341, 823, - 824, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 0, 0, 0, 0, 354, 355, 825, 826, 358, - 359, 827, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 828, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 829, 394, - 395, 396, 830, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 0, 831, 409, 410, 411, 412, - 413, 414, 832, 416, 417, 418, 833, 420, 421, 834, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 835, 437, 836, 439, 0, 440, - 441, 0, 442, 837, 444, 445, 446, 447, 448, 0, - 449, 838, 839, 0, 0, 452, 453, 840, 455, 841, - 1299, 457, 458, 842, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 0, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 1195, 844, 0, 479, 845, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 0, 489, 490, 491, 492, 493, 494, 846, 847, - 848, 849, 850, 851, 852, 853, 854, 855, 856, 506, - 507, 508, 509, 0, 0, 1566, 0, 0, 1567, 0, - 1300, 1301, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 1575, 0, 0, 0, 0, 2112, 0, - 0, 0, 0, 1577, 1566, 0, 0, 1567, 0, 0, - 1578, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1575, 0, 0, 0, 0, 1579, 0, 0, - 0, 0, 1577, 1566, 0, 0, 1567, 0, 0, 1578, - 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1575, 0, 0, 0, 0, 1579, 0, 0, 0, - 0, 1577, 0, 2113, 0, 0, 0, 0, 1578, 0, - 0, 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, - 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1579, 0, 0, 0, 0, - 1575, 0, 0, 0, 1836, 0, 0, 0, 0, 0, - 1577, 0, 0, 1580, 0, 0, 0, 1578, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1581, 0, 0, 0, 0, 1582, 0, 0, 0, 0, - 0, 0, 1872, 0, 1579, 0, 0, 1873, 0, 0, - 0, 0, 1580, 0, 0, 0, 0, 0, 1583, 1584, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1581, - 0, 0, 0, 1585, 1582, 0, 0, 0, 0, 0, - 3427, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1580, 0, 0, 0, 0, 0, 1583, 1584, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1581, 0, - 0, 1586, 1585, 1582, 1587, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1588, 0, - 0, 1589, 0, 0, 0, 0, 1583, 1584, 0, 0, - 1580, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1586, 1585, 0, 1587, 0, 0, 0, 1581, 0, 0, - 0, 0, 1582, 0, 0, 0, 0, 1588, 0, 0, - 1589, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1583, 1584, 0, 0, 1586, - 0, 0, 1587, 0, 0, 0, 0, 0, 0, 0, - 1585, 0, 0, 0, 0, 0, 1588, 0, 0, 1589, + 509, 510, 511, 512, 513, 514, 515, 964, 1339, 815, + 0, 0, 0, 1046, 0, 0, 2688, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 117, 118, 119, 120, + 121, 122, 123, 124, 0, 125, 126, 127, 0, 0, + 0, 560, 0, 0, 0, 0, 565, 129, 130, 0, + 131, 132, 133, 567, 135, 136, 137, 568, 569, 570, + 571, 572, 0, 143, 144, 145, 146, 147, 148, 0, + 0, 149, 150, 151, 152, 576, 577, 155, 0, 156, + 157, 158, 159, 579, 0, 581, 0, 583, 163, 164, + 165, 166, 167, 584, 169, 170, 171, 0, 172, 173, + 174, 175, 176, 177, 0, 587, 179, 180, 181, 182, + 183, 184, 185, 186, 187, 188, 589, 190, 191, 590, + 193, 0, 194, 0, 195, 196, 197, 198, 199, 200, + 0, 0, 201, 202, 203, 204, 0, 0, 205, 206, + 207, 208, 209, 0, 210, 211, 212, 0, 213, 214, + 215, 0, 216, 217, 218, 219, 600, 221, 222, 223, + 224, 225, 601, 1340, 227, 0, 228, 229, 604, 231, + 0, 232, 0, 233, 607, 0, 609, 236, 237, 610, + 611, 240, 0, 241, 0, 614, 615, 244, 245, 0, + 246, 247, 248, 249, 250, 251, 252, 617, 254, 255, + 256, 257, 0, 258, 259, 260, 261, 262, 263, 264, + 0, 265, 620, 621, 268, 269, 270, 271, 272, 622, + 623, 0, 625, 0, 276, 627, 628, 279, 629, 281, + 282, 283, 284, 285, 286, 0, 0, 287, 632, 289, + 633, 0, 291, 292, 293, 294, 295, 296, 297, 298, + 635, 300, 301, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, 314, 315, 316, 317, 318, + 319, 636, 637, 638, 323, 324, 325, 639, 0, 327, + 328, 641, 330, 0, 643, 332, 644, 334, 335, 336, + 0, 337, 338, 1341, 0, 339, 340, 341, 0, 0, + 342, 343, 650, 651, 346, 652, 653, 349, 350, 351, + 352, 353, 354, 355, 356, 357, 358, 359, 0, 0, + 0, 0, 360, 361, 658, 659, 364, 365, 660, 367, + 368, 369, 0, 370, 371, 372, 373, 374, 375, 0, + 376, 377, 378, 663, 380, 381, 382, 383, 0, 384, + 385, 386, 387, 388, 389, 390, 391, 392, 393, 394, + 395, 396, 0, 397, 398, 666, 400, 401, 402, 667, + 404, 405, 406, 407, 408, 409, 410, 411, 412, 413, + 414, 415, 416, 0, 669, 417, 418, 419, 420, 421, + 422, 670, 424, 425, 0, 672, 427, 428, 673, 430, + 0, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 675, 444, 676, 0, 0, 446, 447, + 0, 448, 680, 450, 451, 452, 453, 454, 0, 455, + 682, 683, 0, 0, 458, 459, 686, 461, 687, 1342, + 463, 464, 689, 466, 467, 468, 469, 470, 0, 0, + 471, 472, 473, 0, 474, 475, 476, 477, 0, 478, + 479, 480, 481, 482, 694, 695, 0, 485, 697, 487, + 488, 489, 490, 491, 492, 493, 0, 0, 494, 0, + 0, 495, 496, 497, 498, 499, 500, 702, 703, 704, + 705, 706, 707, 708, 709, 710, 711, 712, 512, 513, + 514, 515, 0, 0, 1607, 0, 0, 1608, 0, 1343, + 1344, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1566, 0, 0, 1567, 1590, 0, 0, 1568, 1569, - 1570, 1571, 1572, 1573, 1574, 0, 0, 3428, 1586, 0, - 0, 1587, 0, 0, 0, 0, 0, 0, 0, 1575, - 0, 0, 0, 0, 0, 1588, 0, 0, 1589, 1577, - 0, 0, 0, 0, 1590, 0, 1578, 0, 0, 0, + 0, 0, 1616, 0, 0, 0, 0, 2161, 0, 0, + 0, 0, 1618, 1607, 0, 0, 1608, 0, 0, 1619, + 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 2116, 1566, - 0, 0, 1567, 1579, 0, 0, 1568, 1569, 1570, 1571, - 1572, 1573, 1574, 1590, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1575, 0, 0, - 0, 1878, 0, 0, 0, 0, 0, 1577, 0, 0, - 0, 0, 1591, 0, 1578, 1592, 1593, 1594, 0, 1595, - 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, 0, - 0, 0, 1590, 0, 0, 0, 1566, 0, 0, 1567, - 0, 1579, 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, - 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, - 1597, 1598, 1599, 1600, 1575, 0, 0, 0, 0, 1580, - 0, 0, 0, 0, 1577, 0, 0, 0, 0, 0, - 0, 1578, 0, 0, 0, 0, 1581, 0, 0, 0, - 1591, 1582, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, - 1598, 1599, 1600, 0, 0, 0, 0, 0, 1579, 0, - 0, 0, 0, 0, 1583, 1584, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 0, 0, 0, 0, 0, 1580, 0, 1591, - 0, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, - 1599, 1600, 0, 0, 1581, 0, 0, 0, 0, 1582, - 0, 0, 0, 0, 0, 0, 0, 1586, 0, 0, - 1587, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1583, 1584, 1588, 0, 0, 1589, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1585, 0, 1883, - 0, 0, 0, 0, 1580, 0, 0, 0, 0, 1843, + 0, 1616, 0, 0, 0, 0, 1620, 0, 0, 0, + 0, 1618, 1607, 0, 0, 1608, 0, 0, 1619, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1581, 0, 0, 0, 0, 1582, 0, 0, 0, - 0, 0, 0, 0, 1566, 1586, 0, 1567, 1587, 0, - 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 1583, - 1584, 0, 1588, 0, 0, 1589, 0, 0, 0, 0, - 0, 0, 1575, 0, 1585, 0, 1885, 0, 0, 0, - 0, 0, 1577, 0, 0, 0, 0, 0, 0, 1578, - 0, 1590, 0, 0, 0, 1566, 0, 0, 1567, 0, - 0, 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, - 0, 0, 1586, 0, 0, 1587, 1579, 0, 0, 0, - 0, 0, 0, 1575, 0, 0, 0, 0, 0, 1588, - 0, 0, 1589, 1577, 0, 0, 0, 0, 0, 0, - 1578, 0, 0, 0, 1566, 0, 0, 1567, 0, 0, - 0, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 1590, - 0, 0, 0, 0, 0, 0, 0, 1579, 0, 0, - 0, 0, 1575, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1577, 1566, 0, 0, 1567, 0, 0, 1578, - 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 1575, 1580, 0, 0, 2699, 1579, 0, 0, 0, - 0, 1577, 0, 0, 0, 0, 1590, 0, 1578, 1581, - 0, 0, 0, 0, 1582, 0, 0, 0, 0, 0, + 1616, 0, 0, 0, 0, 1620, 0, 0, 0, 0, + 1618, 0, 2162, 0, 0, 0, 0, 1619, 0, 0, + 1607, 0, 0, 1608, 0, 0, 0, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1620, 0, 0, 0, 1616, 0, + 0, 0, 1886, 0, 0, 0, 0, 0, 1618, 0, + 1607, 0, 1621, 1608, 0, 1619, 0, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 1622, + 0, 0, 0, 0, 1623, 0, 0, 0, 1616, 0, + 0, 1922, 1620, 0, 0, 0, 1923, 0, 1618, 0, + 0, 1621, 0, 0, 0, 1619, 0, 1624, 1625, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1622, 0, + 0, 0, 1626, 1623, 0, 0, 0, 0, 0, 3500, + 0, 0, 1620, 0, 0, 0, 0, 0, 0, 0, + 1621, 0, 0, 0, 0, 0, 1624, 1625, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 1627, 1626, 1623, 1628, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1629, 0, 0, + 1630, 0, 0, 0, 0, 1624, 1625, 0, 1621, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1627, + 1626, 0, 1628, 0, 0, 1622, 0, 0, 0, 0, + 1623, 0, 0, 0, 0, 0, 1629, 0, 0, 1630, + 0, 0, 0, 0, 0, 0, 0, 0, 1621, 0, + 0, 0, 0, 1624, 1625, 0, 0, 0, 1627, 0, + 0, 1628, 0, 0, 0, 1622, 0, 0, 1626, 0, + 1623, 0, 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 2007, 0, 1579, 0, 1583, 1584, 0, - 0, 0, 0, 1580, 0, 0, 1591, 0, 0, 1592, - 1593, 1594, 1585, 1595, 1596, 1597, 1598, 1599, 1600, 0, - 1581, 0, 0, 1566, 0, 1582, 1567, 0, 0, 0, - 1568, 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1583, 1584, - 1586, 1575, 1580, 1587, 0, 0, 0, 0, 0, 0, - 0, 1577, 0, 1585, 0, 0, 0, 1588, 1578, 1581, - 1589, 0, 0, 1591, 1582, 0, 1592, 1593, 1594, 0, - 1595, 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, - 0, 1580, 0, 0, 0, 1579, 0, 1583, 1584, 0, - 0, 1586, 0, 0, 1587, 0, 0, 0, 1581, 0, - 0, 0, 1585, 1582, 0, 0, 0, 0, 1588, 0, - 0, 1589, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1583, 1584, 0, 0, + 0, 0, 0, 1624, 1625, 0, 0, 1631, 0, 0, + 0, 0, 0, 0, 0, 0, 1627, 3501, 1626, 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1586, 1585, 0, 1587, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 1590, 0, 0, 1588, 0, 0, - 1589, 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, - 1569, 1570, 1571, 1572, 1573, 1574, 0, 0, 0, 1586, - 0, 1580, 1587, 0, 0, 0, 0, 0, 0, 0, - 1575, 0, 0, 0, 0, 0, 1588, 0, 1581, 1589, - 1577, 0, 0, 1582, 0, 1590, 0, 1578, 0, 0, - 0, 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, - 0, 0, 1572, 1573, 1574, 0, 1583, 1584, 0, 0, - 0, 0, 0, 0, 1579, 0, 0, 0, 0, 1575, - 0, 1585, 0, 0, 0, 0, 0, 0, 0, 1577, - 0, 0, 0, 0, 1590, 0, 1578, 0, 0, 0, - 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, - 1597, 1598, 1599, 1600, 0, 0, 0, 0, 0, 1586, - 0, 0, 1587, 1579, 0, 0, 0, 2686, 0, 0, - 0, 0, 0, 1590, 0, 0, 1588, 0, 0, 1589, + 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1631, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1627, 0, 0, 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, - 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, 0, - 1580, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 1566, 1581, 0, 1567, - 0, 0, 1582, 1568, 1569, 0, 0, 1572, 1573, 1574, - 0, 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, - 1597, 1598, 1599, 1600, 1575, 1583, 1584, 0, 0, 1580, - 0, 0, 0, 0, 1577, 0, 0, 0, 0, 0, - 1585, 1578, 0, 1590, 0, 0, 1581, 0, 0, 0, - 1591, 1582, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, - 1598, 1599, 1600, 0, 0, 0, 0, 0, 1579, 0, - 0, 0, 0, 0, 1583, 1584, 0, 0, 1586, 0, - 0, 1587, 0, 0, 0, 0, 0, 0, 0, 1585, - 0, 0, 0, 0, 0, 1588, 0, 0, 1589, 0, - 1566, 0, 0, 1567, 0, 0, 0, 1568, 1569, 0, - 0, 1572, 1573, 1574, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 1586, 0, 0, - 1587, 0, 0, 0, 0, 0, 0, 0, 1577, 0, - 0, 0, 0, 0, 1588, 1578, 0, 1589, 0, 0, - 1591, 0, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, - 1598, 1599, 1600, 0, 1580, 0, 0, 0, 0, 0, - 0, 0, 1579, 0, 0, 0, 0, 1566, 0, 0, - 1567, 1581, 0, 0, 1568, 1569, 1582, 0, 1572, 1573, - 1574, 0, 1590, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1583, - 1584, 0, 0, 0, 0, 1577, 0, 0, 0, 0, - 0, 0, 1578, 0, 1585, 0, 0, 0, 0, 0, + 2165, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1893, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 1590, 0, 0, 0, 0, 0, 0, 0, 1579, + 0, 0, 0, 0, 1632, 0, 0, 1633, 1634, 1635, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 0, + 0, 0, 0, 1631, 0, 0, 0, 0, 1607, 0, + 0, 1608, 0, 0, 0, 1609, 1610, 1611, 1612, 1613, + 1614, 1615, 0, 1632, 0, 0, 1633, 1634, 1635, 0, + 1636, 1637, 1638, 1639, 1640, 1641, 1616, 0, 0, 0, + 1928, 0, 0, 1631, 0, 0, 1618, 0, 0, 0, + 0, 0, 0, 1619, 0, 0, 0, 0, 0, 0, + 0, 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 0, 0, 0, 0, 0, + 1620, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1607, 0, 0, 1608, 0, + 0, 0, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, + 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 1616, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 1618, 0, 0, 0, 0, 0, 0, + 1619, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, + 1639, 1640, 1641, 0, 0, 0, 0, 1620, 0, 0, + 0, 0, 1607, 0, 0, 1608, 1621, 0, 0, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, + 0, 0, 0, 1622, 0, 0, 0, 0, 1623, 0, + 1616, 0, 0, 0, 1935, 0, 0, 0, 0, 0, + 1618, 0, 0, 0, 0, 0, 0, 1619, 0, 0, + 0, 1624, 1625, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1626, 0, 0, 0, + 0, 0, 0, 0, 1620, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1933, 1607, + 0, 0, 1608, 1621, 0, 0, 1609, 1610, 1611, 1612, + 1613, 1614, 1615, 0, 1627, 0, 0, 1628, 0, 0, + 1622, 0, 0, 0, 0, 1623, 0, 1616, 0, 0, + 0, 1629, 0, 0, 1630, 0, 0, 1618, 0, 0, + 0, 0, 0, 0, 1619, 0, 0, 0, 1624, 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 1586, 0, 0, 1587, 0, 0, 1580, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 1588, - 0, 0, 0, 0, 0, 1581, 0, 0, 0, 0, - 1582, 0, 0, 0, 0, 0, 0, 0, 0, 1591, - 0, 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, - 2133, 1600, 0, 1583, 1584, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1585, 0, + 0, 0, 0, 1626, 0, 0, 0, 0, 0, 0, + 0, 1620, 0, 0, 0, 0, 0, 0, 0, 0, + 1621, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 0, 1627, 1623, 0, 1628, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1629, 0, + 0, 1630, 0, 0, 0, 1624, 1625, 0, 0, 0, + 0, 1631, 0, 0, 1607, 0, 0, 1608, 0, 0, + 1626, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 2056, 0, 0, + 0, 0, 1616, 0, 0, 0, 0, 1621, 0, 0, + 0, 0, 1618, 0, 0, 0, 0, 0, 1627, 1619, + 0, 1628, 0, 0, 1622, 0, 0, 0, 0, 1623, + 0, 0, 0, 0, 0, 1629, 0, 0, 1630, 0, + 0, 0, 0, 0, 0, 0, 1620, 0, 0, 0, + 0, 0, 1624, 1625, 0, 0, 0, 0, 1631, 0, + 0, 0, 0, 0, 0, 0, 0, 1626, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1632, 0, + 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, + 1641, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1627, 0, 0, 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1580, 0, 0, 1591, 0, - 0, 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, - 1600, 0, 1581, 0, 0, 0, 1586, 1582, 0, 1587, - 0, 0, 0, 0, 0, 0, 1590, 0, 0, 0, - 0, 0, 0, 1588, 0, 0, 0, 0, 0, 0, - -2003, -2003, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 1585, 0, 0, 0, 0, + 0, 0, 1629, 0, 0, 1630, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 0, 1621, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1622, + 0, 0, 0, 0, 1623, 1632, 0, 0, 1633, 1634, + 1635, 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1624, 1625, 0, + 0, 0, 0, 0, 0, 0, 1607, 0, 0, 1608, + 0, 0, 1626, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 1631, 0, 1616, 0, 0, 0, 2762, 0, + 0, 0, 0, 0, 1618, 0, 0, 0, 0, 0, + 1627, 1619, 1632, 1628, 0, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 1640, 1641, 0, 0, 1629, 0, 0, + 1630, 0, 0, 0, 0, 0, 0, 0, 1620, 0, + 1607, 0, 0, 1608, 0, 0, 0, 1609, 1610, 1611, + 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1616, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1618, 0, + 0, 0, 0, 0, 0, 1619, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1632, + 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, + 1640, 1641, 1620, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1631, 0, 1607, + 0, 0, 1608, 0, 1621, 0, 1609, 1610, 1611, 1612, + 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, + 0, 1622, 0, 0, 0, 0, 1623, 1616, 0, 0, + 2749, 0, 0, 0, 0, 0, 0, 1618, 0, 0, + 0, 0, 0, 0, 1619, 0, 0, 0, 0, 1624, + 1625, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1626, 0, 0, 0, 0, 0, + 0, 1620, 0, 0, 0, 0, 0, 0, 1621, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1622, 0, 0, 0, 0, + 1623, 0, 1627, 0, 1632, 1628, 0, 1633, 1634, 1635, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 0, 0, 1629, + 0, 0, 1630, 1624, 1625, 0, 0, 0, 0, 0, + 0, 0, 1607, 0, 0, 1608, 0, 0, 1626, 1609, + 1610, 1611, 1612, 1613, 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, -2003, 0, 0, 0, + 1616, 0, 0, 0, 0, 0, 0, 1621, 0, 0, + 1618, 0, 0, 0, 0, 0, 1627, 1619, 0, 1628, + 0, 0, 0, 0, 1622, 0, 0, 0, 0, 1623, + 0, 0, 0, 1629, 0, 0, 1630, 0, 0, 0, + 0, 0, 0, 0, 1620, 0, 0, 0, 0, 0, + 0, 0, 1802, 1625, 0, 0, 0, 0, 0, 1631, + 0, 0, 0, 0, 0, 0, 0, 1626, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 1607, 0, + 0, 1608, 0, 0, 0, 1609, 1610, 0, 0, 1613, + 1614, 1615, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 1627, 1616, 0, 1628, 0, + 0, 0, 0, 0, 0, 0, 1618, 0, 0, 0, + 0, 0, 1629, 1619, 0, 1630, 0, 0, 0, 0, + 0, 0, 0, 1631, 1607, 0, 0, 1608, 0, 0, + 1621, 1609, 1610, 0, 0, -2041, -2041, -2041, 0, 0, + 1620, 0, 0, 0, 0, 0, 0, 1622, 0, 0, + 0, 0, 1623, 0, 0, 0, 1632, 0, 0, 1633, + 1634, 1635, 1618, 1636, 1637, 1638, 1639, 1640, 1641, 1619, + 0, 1607, 0, 0, 1608, 1624, 1625, 0, 1609, 1610, + 0, 0, 1613, 1614, 1615, 0, 0, 0, 0, 0, + 1626, 0, 0, 0, 0, 0, 1620, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1618, + 0, 1607, 1631, 0, 1608, 0, 1619, 0, 1609, 1610, + 0, 0, 1613, 1614, 1615, 0, 0, 0, 1627, 0, + 1632, 1628, 0, 1633, 1634, 1635, 1621, 1636, 1637, 1638, + 1639, 1640, 1641, 1620, 0, 1629, 0, 0, 1630, 1618, + 0, 0, 0, 1622, 0, 0, 1619, 0, 1623, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1588, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 1590, 0, 0, 1591, 0, 0, 1592, 1593, 1594, 0, - 1595, 1596, 1597, 1598, 1599, 1600, 0, 0, 0, 0, + 0, 1624, 1625, 1620, 0, 0, 0, 0, 0, 0, + 0, 0, 1621, 0, 0, 0, 1626, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1622, + 0, 0, 0, 0, 1623, 0, 0, 0, 0, 1632, + 0, 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, + 1640, 1641, 0, 0, 1627, 0, 0, 1628, 0, 1621, + 0, 0, 0, 0, 0, 1631, 0, 0, 0, 0, + 0, 1629, 1626, 0, 1630, 0, 1622, 0, 0, 0, + 0, 1623, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 1621, + 0, 0, 0, 0, 1624, 1625, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 1622, 0, 0, 1626, + 0, 1623, 0, 0, 0, 0, 0, 1629, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, -2041, -2041, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1627, 0, 1626, + 1628, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 1631, 1632, 0, 1629, 1633, 1634, 1635, 0, 1636, + 1637, 1638, 1639, 2182, 1641, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + -2041, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1629, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 1631, 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, 1590, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1631, 0, 0, 0, 1632, 0, + 0, 1633, 1634, 1635, 0, 1636, 1637, 1638, 1639, 1640, + 1641, 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, 1591, 0, 0, - 1592, 1593, 1594, 0, 1595, 1596, 1597, 1598, 1599, 1600, + 0, 0, 0, 0, 1631, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 1632, 0, 0, -2041, -2041, -2041, + 0, 1636, 1637, 1638, 1639, 1640, 1641, 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, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, + 1638, 1639, 1640, 1641, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 1127, 0, - 2030, 0, 0, 0, 1591, 0, 0, 1592, 1593, 1594, - 0, 1595, 1596, 1597, 1598, 1599, 1600, 111, 112, 113, - 114, 115, 116, 117, 118, 1128, 119, 120, 121, 1129, - 1130, 1131, 776, 1132, 1133, 1134, 1135, 777, 123, 124, - 1136, 125, 126, 127, 778, 129, 130, 131, 779, 780, - 781, 782, 783, 1137, 137, 138, 139, 140, 141, 142, - 1138, 1139, 143, 144, 145, 146, 784, 785, 149, 1140, - 150, 151, 152, 153, 786, 1141, 787, 1142, 788, 157, - 158, 159, 160, 161, 789, 163, 164, 165, 1143, 166, - 167, 168, 169, 170, 171, 1144, 790, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 791, 184, 185, - 792, 187, 1145, 188, 1146, 189, 190, 191, 192, 193, - 194, 1147, 195, 196, 197, 198, 199, 1148, 1149, 200, - 201, 202, 203, 204, 1150, 205, 206, 207, 1151, 208, - 209, 210, 1152, 211, 212, 213, 214, 793, 216, 217, - 218, 219, 220, 794, 1153, 222, 1154, 223, 224, 795, - 226, 1155, 227, 1156, 228, 796, 1157, 797, 231, 232, - 798, 799, 235, 1158, 236, 1159, 800, 801, 239, 240, - 1160, 241, 242, 243, 244, 245, 246, 247, 802, 249, - 250, 251, 252, 1161, 253, 254, 255, 256, 257, 258, - 259, 1162, 260, 803, 804, 263, 264, 265, 266, 267, - 805, 806, 1163, 807, 1164, 271, 808, 809, 274, 810, - 276, 277, 278, 279, 280, 281, 1165, 1166, 282, 811, - 284, 812, 1167, 286, 287, 288, 289, 290, 291, 292, - 293, 813, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 814, 815, 816, 318, 319, 320, 817, 1168, - 322, 323, 818, 325, 1169, 819, 327, 820, 329, 330, - 331, 1170, 332, 333, 1171, 1172, 334, 335, 336, 1173, - 1174, 337, 338, 821, 822, 341, 823, 824, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 1175, 1176, - 1177, 1178, 354, 355, 825, 826, 358, 359, 827, 361, - 362, 363, 1179, 364, 365, 366, 367, 368, 369, 1180, - 370, 371, 372, 828, 374, 375, 376, 377, 1181, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 1182, 391, 392, 829, 394, 395, 396, 830, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 1183, 831, 409, 410, 411, 412, 413, 414, 832, - 416, 417, 418, 833, 420, 421, 834, 423, 1184, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 835, 437, 836, 439, 1185, 440, 441, 1186, 442, - 837, 444, 445, 446, 447, 448, 1187, 449, 838, 839, - 1188, 1189, 452, 453, 840, 455, 841, 1190, 457, 458, - 842, 460, 461, 462, 463, 464, 1191, 1192, 465, 466, - 467, 1193, 468, 469, 470, 471, 1194, 472, 473, 474, - 475, 476, 1195, 844, 1196, 479, 845, 481, 482, 483, - 484, 485, 486, 487, 1197, 1198, 488, 1199, 1200, 489, - 490, 491, 492, 493, 494, 846, 847, 848, 849, 850, - 851, 852, 853, 854, 855, 856, 506, 507, 508, 509, - 1127, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 1128, 119, 120, - 121, 1129, 1130, 1131, 776, 1132, 1133, 1134, 1135, 777, - 123, 124, 1136, 125, 126, 127, 778, 129, 130, 131, - 779, 780, 781, 782, 783, 1137, 137, 138, 139, 140, - 141, 142, 1138, 1139, 143, 144, 145, 146, 784, 785, - 149, 1140, 150, 151, 152, 153, 786, 1141, 787, 1142, - 788, 157, 158, 159, 160, 161, 789, 163, 164, 165, - 1143, 166, 167, 168, 169, 170, 171, 1144, 790, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 791, - 184, 185, 792, 187, 1145, 188, 1146, 189, 190, 191, - 192, 193, 194, 1147, 195, 196, 197, 198, 199, 1148, - 1149, 200, 201, 202, 203, 204, 1150, 205, 206, 207, - 1151, 208, 209, 210, 1152, 211, 212, 213, 214, 793, - 216, 217, 218, 219, 220, 794, 1153, 222, 1154, 223, - 224, 795, 226, 1155, 227, 1156, 228, 796, 1157, 797, - 231, 232, 798, 799, 235, 1158, 236, 1159, 800, 801, - 239, 240, 1160, 241, 242, 243, 244, 245, 246, 247, - 802, 249, 250, 251, 252, 1161, 253, 254, 255, 256, - 257, 258, 259, 1162, 260, 803, 804, 263, 264, 265, - 266, 267, 805, 806, 1163, 807, 1164, 271, 808, 809, - 274, 810, 276, 277, 278, 279, 280, 281, 1165, 1166, - 282, 811, 284, 812, 1167, 286, 287, 288, 289, 290, - 291, 292, 293, 813, 295, 296, 297, 298, 299, 300, + 0, 0, 0, 0, 0, 555, 0, 2079, 0, 0, + 0, 1632, 0, 0, 1633, 1634, 1635, 0, 1636, 1637, + 1638, 1639, 1640, 1641, 117, 118, 119, 120, 121, 122, + 123, 124, 556, 125, 126, 127, 557, 558, 559, 560, + 561, 562, 563, 564, 565, 129, 130, 566, 131, 132, + 133, 567, 135, 136, 137, 568, 569, 570, 571, 572, + 573, 143, 144, 145, 146, 147, 148, 574, 575, 149, + 150, 151, 152, 576, 577, 155, 578, 156, 157, 158, + 159, 579, 580, 581, 582, 583, 163, 164, 165, 166, + 167, 584, 169, 170, 171, 585, 172, 173, 174, 175, + 176, 177, 586, 587, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 589, 190, 191, 590, 193, 591, + 194, 592, 195, 196, 197, 198, 199, 200, 593, 594, + 201, 202, 203, 204, 595, 596, 205, 206, 207, 208, + 209, 597, 210, 211, 212, 598, 213, 214, 215, 599, + 216, 217, 218, 219, 600, 221, 222, 223, 224, 225, + 601, 602, 227, 603, 228, 229, 604, 231, 605, 232, + 606, 233, 607, 608, 609, 236, 237, 610, 611, 240, + 612, 241, 613, 614, 615, 244, 245, 616, 246, 247, + 248, 249, 250, 251, 252, 617, 254, 255, 256, 257, + 618, 258, 259, 260, 261, 262, 263, 264, 619, 265, + 620, 621, 268, 269, 270, 271, 272, 622, 623, 624, + 625, 626, 276, 627, 628, 279, 629, 281, 282, 283, + 284, 285, 286, 630, 631, 287, 632, 289, 633, 634, + 291, 292, 293, 294, 295, 296, 297, 298, 635, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 814, 815, 816, 318, 319, 320, - 817, 1168, 322, 323, 818, 325, 1169, 819, 327, 820, - 329, 330, 331, 1170, 332, 333, 1171, 1172, 334, 335, - 336, 1173, 1174, 337, 338, 821, 822, 341, 823, 824, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 1175, 1176, 1177, 1178, 354, 355, 825, 826, 358, 359, - 827, 361, 362, 363, 1179, 364, 365, 366, 367, 368, - 369, 1180, 370, 371, 372, 828, 374, 375, 376, 377, - 1181, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 1182, 391, 392, 829, 394, 395, - 396, 830, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 1183, 831, 409, 410, 411, 412, 413, - 414, 832, 416, 417, 418, 833, 420, 421, 834, 423, - 1184, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 835, 437, 836, 439, 1185, 440, 441, - 1186, 442, 837, 444, 445, 446, 447, 448, 1187, 449, - 838, 839, 1188, 1189, 452, 453, 840, 455, 841, 1190, - 457, 458, 842, 460, 461, 462, 463, 464, 1191, 1192, - 465, 466, 467, 1193, 468, 469, 470, 471, 1194, 472, - 473, 474, 475, 476, 1195, 844, 1196, 479, 845, 481, - 482, 483, 484, 485, 486, 487, 1197, 1198, 488, 1199, - 1200, 489, 490, 491, 492, 493, 494, 846, 847, 848, - 849, 850, 851, 852, 853, 854, 855, 856, 506, 507, - 508, 509, 1127, 0, 0, 0, 0, 0, 0, 0, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 636, + 637, 638, 323, 324, 325, 639, 640, 327, 328, 641, + 330, 642, 643, 332, 644, 334, 335, 336, 645, 337, + 338, 646, 647, 339, 340, 341, 648, 649, 342, 343, + 650, 651, 346, 652, 653, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 654, 655, 656, 657, + 360, 361, 658, 659, 364, 365, 660, 367, 368, 369, + 661, 370, 371, 372, 373, 374, 375, 662, 376, 377, + 378, 663, 380, 381, 382, 383, 664, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 665, 397, 398, 666, 400, 401, 402, 667, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 668, 669, 417, 418, 419, 420, 421, 422, 670, + 424, 425, 671, 672, 427, 428, 673, 430, 674, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 675, 444, 676, 677, 678, 446, 447, 679, 448, + 680, 450, 451, 452, 453, 454, 681, 455, 682, 683, + 684, 685, 458, 459, 686, 461, 687, 688, 463, 464, + 689, 466, 467, 468, 469, 470, 690, 691, 471, 472, + 473, 692, 474, 475, 476, 477, 693, 478, 479, 480, + 481, 482, 694, 695, 696, 485, 697, 487, 488, 489, + 490, 491, 492, 493, 698, 699, 494, 700, 701, 495, + 496, 497, 498, 499, 500, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 512, 513, 514, 515, + 555, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 556, 125, 126, + 127, 557, 558, 559, 560, 561, 562, 563, 564, 565, + 129, 130, 566, 131, 132, 133, 567, 135, 136, 137, + 568, 569, 570, 571, 572, 573, 143, 144, 145, 146, + 147, 148, 574, 575, 149, 150, 151, 152, 576, 577, + 155, 578, 156, 157, 158, 159, 579, 580, 581, 582, + 583, 163, 164, 165, 166, 167, 584, 169, 170, 171, + 585, 172, 173, 174, 175, 176, 177, 586, 587, 179, + 180, 181, 182, 183, 184, 588, 186, 187, 188, 589, + 190, 191, 590, 193, 591, 194, 592, 195, 196, 197, + 198, 199, 200, 593, 594, 201, 202, 203, 204, 595, + 596, 205, 206, 207, 208, 209, 597, 210, 211, 212, + 598, 213, 214, 215, 599, 216, 217, 218, 219, 600, + 221, 222, 223, 224, 225, 601, 602, 227, 603, 228, + 229, 604, 231, 605, 232, 606, 233, 607, 608, 609, + 236, 237, 610, 611, 240, 612, 241, 613, 614, 615, + 244, 245, 616, 246, 247, 248, 249, 250, 251, 252, + 617, 254, 255, 256, 257, 618, 258, 259, 260, 261, + 262, 263, 264, 619, 265, 620, 621, 268, 269, 270, + 271, 272, 622, 623, 624, 625, 626, 276, 627, 628, + 279, 629, 281, 282, 283, 284, 285, 286, 630, 631, + 287, 632, 289, 633, 634, 291, 292, 293, 294, 295, + 296, 297, 298, 635, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 636, 637, 638, 323, 324, 325, + 639, 640, 327, 328, 641, 330, 642, 643, 332, 644, + 334, 335, 336, 645, 337, 338, 646, 647, 339, 340, + 341, 648, 649, 342, 343, 650, 651, 346, 652, 653, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 654, 655, 656, 657, 360, 361, 658, 659, 364, + 365, 660, 367, 368, 369, 661, 370, 371, 372, 373, + 374, 375, 662, 376, 377, 378, 663, 380, 381, 382, + 383, 664, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 665, 397, 398, 666, 400, + 401, 402, 667, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 668, 669, 417, 418, + 419, 420, 421, 422, 670, 424, 425, 671, 672, 427, + 428, 673, 430, 674, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 675, 444, 676, 677, + 678, 446, 447, 679, 448, 680, 450, 451, 452, 453, + 454, 681, 455, 682, 683, 684, 685, 458, 459, 686, + 461, 687, 688, 463, 464, 689, 466, 467, 468, 469, + 470, 690, 691, 471, 472, 473, 692, 474, 475, 476, + 477, 693, 478, 479, 480, 481, 482, 694, 695, 696, + 485, 697, 487, 488, 489, 490, 491, 492, 493, 698, + 699, 494, 700, 701, 495, 496, 497, 498, 499, 500, + 702, 703, 704, 705, 706, 707, 708, 709, 710, 711, + 712, 512, 513, 514, 515, 555, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 2178, 116, 117, 118, 1128, - 119, 120, 121, 1129, 1130, 1131, 776, 1132, 1133, 1134, - 1135, 777, 123, 124, 1136, 125, 126, 127, 778, 129, - 130, 131, 779, 780, 781, 782, 783, 1137, 137, 138, - 139, 140, 141, 142, 1138, 1139, 143, 144, 145, 146, - 784, 785, 149, 1140, 150, 151, 152, 153, 786, 1141, - 787, 1142, 788, 157, 158, 159, 160, 161, 789, 163, - 164, 165, 1143, 166, 167, 168, 169, 170, 171, 1144, - 790, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 791, 184, 185, 792, 187, 1145, 188, 1146, 189, - 190, 191, 192, 193, 194, 1147, 195, 196, 197, 198, - 199, 1148, 1149, 200, 201, 202, 2179, 204, 1150, 205, - 206, 207, 1151, 208, 209, 210, 1152, 211, 212, 213, - 214, 793, 216, 217, 218, 219, 220, 794, 1153, 222, - 1154, 223, 224, 795, 226, 1155, 227, 1156, 228, 796, - 1157, 797, 231, 232, 798, 799, 235, 1158, 236, 1159, - 800, 801, 239, 240, 1160, 241, 242, 243, 244, 245, - 246, 247, 802, 249, 250, 251, 252, 1161, 253, 254, - 255, 256, 257, 258, 259, 1162, 260, 803, 804, 263, - 264, 265, 266, 267, 805, 806, 1163, 807, 1164, 271, - 808, 809, 274, 810, 276, 277, 278, 279, 280, 281, - 1165, 1166, 282, 811, 284, 812, 1167, 286, 287, 288, - 289, 290, 291, 292, 293, 813, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 814, 815, 816, 318, - 319, 320, 817, 1168, 322, 323, 818, 325, 1169, 819, - 327, 820, 329, 330, 331, 1170, 332, 333, 1171, 1172, - 334, 335, 336, 1173, 1174, 337, 338, 821, 822, 341, - 823, 824, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 1175, 1176, 1177, 1178, 354, 355, 825, 826, - 358, 359, 827, 361, 362, 363, 1179, 364, 365, 366, - 367, 368, 369, 1180, 370, 371, 372, 828, 374, 375, - 376, 377, 1181, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 1182, 391, 392, 829, - 394, 395, 396, 830, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 1183, 831, 409, 410, 411, - 412, 413, 2180, 832, 416, 417, 418, 833, 420, 421, - 834, 423, 1184, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 835, 437, 836, 439, 1185, - 440, 441, 1186, 442, 837, 444, 445, 446, 447, 448, - 1187, 449, 838, 839, 1188, 1189, 452, 453, 840, 455, - 841, 1190, 457, 458, 842, 460, 461, 462, 463, 464, - 1191, 1192, 465, 466, 467, 1193, 468, 469, 470, 471, - 1194, 472, 473, 474, 475, 476, 1195, 844, 1196, 479, - 845, 481, 482, 483, 484, 485, 486, 487, 1197, 1198, - 488, 1199, 1200, 489, 490, 491, 492, 493, 494, 846, - 847, 848, 849, 850, 851, 852, 853, 854, 855, 856, - 506, 507, 508, 509, 775, 0, 642, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 556, 125, 126, 127, 557, 558, 559, 560, + 561, 562, 563, 564, 565, 129, 130, 566, 131, 132, + 133, 567, 135, 136, 137, 568, 569, 570, 571, 572, + 573, 143, 144, 145, 146, 147, 148, 574, 575, 149, + 150, 151, 152, 576, 577, 155, 578, 156, 157, 158, + 159, 579, 580, 581, 582, 583, 163, 164, 165, 166, + 167, 584, 169, 170, 171, 585, 172, 173, 174, 175, + 176, 177, 586, 587, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 589, 190, 191, 590, 193, 591, + 194, 592, 195, 196, 197, 198, 199, 200, 593, 594, + 201, 202, 203, 204, 595, 596, 205, 206, 207, 208, + 209, 597, 210, 211, 212, 598, 213, 214, 215, 599, + 216, 217, 218, 219, 600, 221, 222, 223, 224, 225, + 601, 602, 227, 603, 228, 229, 604, 231, 605, 232, + 606, 233, 607, 608, 609, 236, 237, 610, 611, 240, + 612, 241, 613, 614, 615, 244, 245, 616, 246, 247, + 248, 249, 250, 940, 252, 617, 254, 255, 256, 257, + 618, 258, 259, 260, 261, 262, 263, 264, 619, 265, + 620, 621, 268, 269, 270, 271, 272, 622, 623, 624, + 625, 626, 276, 627, 628, 279, 629, 281, 282, 283, + 284, 285, 286, 630, 631, 287, 632, 289, 633, 634, + 291, 292, 293, 294, 295, 296, 297, 298, 635, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 636, + 637, 638, 323, 324, 325, 639, 640, 327, 328, 641, + 330, 642, 643, 332, 644, 334, 335, 336, 645, 337, + 338, 646, 647, 339, 340, 341, 648, 649, 342, 343, + 650, 651, 346, 652, 653, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 654, 655, 656, 657, + 360, 361, 658, 659, 364, 365, 660, 367, 368, 369, + 661, 370, 371, 372, 373, 374, 375, 662, 376, 377, + 378, 663, 380, 381, 382, 383, 664, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 665, 397, 398, 666, 400, 401, 402, 667, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 668, 669, 417, 418, 419, 420, 421, 422, 670, + 424, 425, 671, 672, 427, 428, 673, 430, 674, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 675, 444, 676, 677, 678, 446, 447, 679, 448, + 680, 450, 451, 452, 453, 454, 681, 455, 682, 683, + 684, 685, 458, 459, 686, 461, 687, 688, 463, 464, + 689, 466, 467, 468, 469, 470, 690, 691, 471, 472, + 473, 692, 474, 475, 476, 477, 693, 478, 479, 480, + 481, 482, 694, 695, 696, 485, 697, 487, 488, 489, + 490, 491, 492, 493, 698, 699, 494, 700, 701, 495, + 496, 497, 498, 499, 500, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 512, 513, 514, 515, + 555, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 556, 125, 126, + 127, 557, 558, 559, 560, 561, 562, 563, 564, 565, + 129, 130, 566, 131, 132, 133, 567, 135, 136, 137, + 568, 569, 570, 571, 572, 573, 143, 144, 145, 146, + 147, 148, 574, 575, 149, 150, 151, 152, 576, 577, + 155, 578, 156, 157, 158, 159, 579, 580, 581, 582, + 583, 163, 164, 165, 166, 167, 584, 169, 170, 171, + 585, 172, 173, 174, 175, 176, 177, 586, 587, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 589, + 190, 191, 590, 193, 591, 194, 592, 195, 196, 197, + 198, 199, 200, 593, 594, 201, 202, 203, 204, 595, + 596, 205, 206, 207, 208, 209, 597, 210, 211, 212, + 598, 213, 214, 215, 599, 216, 217, 218, 219, 600, + 221, 222, 223, 224, 225, 601, 602, 227, 603, 228, + 229, 604, 231, 605, 232, 606, 233, 607, 608, 609, + 236, 237, 610, 611, 240, 612, 241, 613, 614, 615, + 244, 245, 616, 246, 247, 248, 249, 250, 251, 252, + 617, 254, 255, 256, 257, 618, 258, 259, 260, 261, + 262, 263, 264, 619, 265, 620, 621, 268, 269, 270, + 271, 272, 622, 623, 624, 625, 626, 276, 627, 628, + 279, 629, 281, 282, 283, 284, 285, 286, 630, 631, + 287, 632, 289, 633, 634, 291, 292, 293, 294, 295, + 296, 297, 298, 635, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 636, 637, 638, 323, 324, 325, + 639, 640, 327, 328, 641, 330, 642, 643, 332, 644, + 334, 335, 336, 645, 337, 338, 646, 647, 339, 340, + 341, 648, 649, 342, 343, 650, 651, 346, 652, 653, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 654, 655, 656, 657, 360, 361, 658, 659, 364, + 365, 660, 367, 368, 369, 661, 370, 371, 372, 373, + 374, 375, 662, 376, 377, 378, 663, 380, 381, 382, + 383, 664, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 665, 397, 398, 666, 400, + 401, 402, 667, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 668, 669, 417, 418, + 419, 420, 421, 422, 670, 424, 425, 671, 672, 427, + 428, 673, 430, 674, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 675, 444, 676, 677, + 678, 446, 447, 679, 448, 680, 450, 451, 452, 453, + 454, 681, 455, 682, 683, 684, 685, 458, 459, 686, + 461, 687, 688, 463, 464, 689, 466, 467, 468, 469, + 470, 690, 691, 471, 472, 473, 692, 474, 475, 476, + 477, 693, 478, 479, 480, 481, 482, 694, 695, 696, + 485, 697, 487, 488, 489, 490, 491, 492, 493, 698, + 699, 494, 700, 701, 495, 496, 497, 498, 499, 500, + 702, 703, 704, 705, 706, 707, 708, 709, 710, 711, + 712, 512, 513, 514, 515, 555, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 776, 0, - 0, 0, 0, 777, 123, 124, 0, 125, 126, 127, - 778, 129, 130, 131, 779, 780, 781, 782, 783, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 784, 785, 149, 0, 150, 151, 152, 153, - 786, 0, 787, 0, 788, 157, 158, 159, 160, 161, - 789, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 790, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 791, 184, 185, 792, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 793, 216, 217, 218, 219, 220, 794, - 1297, 222, 0, 223, 224, 795, 226, 0, 227, 0, - 228, 796, 0, 797, 231, 232, 798, 799, 235, 0, - 236, 0, 800, 801, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 802, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 803, - 804, 263, 264, 265, 266, 267, 805, 806, 0, 807, - 0, 271, 808, 809, 274, 810, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 811, 284, 812, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 813, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 814, 815, - 816, 318, 319, 320, 817, 0, 322, 323, 818, 325, - 0, 819, 327, 820, 329, 330, 331, 0, 332, 333, - 1298, 0, 334, 335, 336, 0, 0, 337, 338, 821, - 822, 341, 823, 824, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 825, 826, 358, 359, 827, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 828, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 829, 394, 395, 396, 830, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 831, 409, - 410, 411, 412, 413, 414, 832, 416, 417, 418, 833, - 420, 421, 834, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 835, 437, 836, - 439, 0, 440, 441, 0, 442, 837, 444, 445, 446, - 447, 448, 0, 449, 838, 839, 0, 0, 452, 453, - 840, 455, 841, 1299, 457, 458, 842, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 1195, 844, - 0, 479, 845, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 846, 847, 848, 849, 850, 851, 852, 853, 854, - 855, 856, 506, 507, 508, 509, 775, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 2227, 122, + 123, 124, 556, 125, 126, 127, 557, 558, 559, 560, + 561, 562, 563, 564, 565, 129, 130, 566, 131, 132, + 133, 567, 135, 136, 137, 568, 569, 570, 571, 572, + 573, 143, 144, 145, 146, 147, 148, 574, 575, 149, + 150, 151, 152, 576, 577, 155, 578, 156, 157, 158, + 159, 579, 580, 581, 582, 583, 163, 164, 165, 166, + 167, 584, 169, 170, 171, 585, 172, 173, 174, 175, + 176, 177, 586, 587, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 589, 190, 191, 590, 193, 591, + 194, 592, 195, 196, 197, 198, 199, 200, 593, 594, + 201, 202, 203, 204, 595, 596, 205, 206, 207, 2228, + 209, 597, 210, 211, 212, 598, 213, 214, 215, 599, + 216, 217, 218, 219, 600, 221, 222, 223, 224, 225, + 601, 602, 227, 603, 228, 229, 604, 231, 605, 232, + 606, 233, 607, 608, 609, 236, 237, 610, 611, 240, + 612, 241, 613, 614, 615, 244, 245, 616, 246, 247, + 248, 249, 250, 251, 252, 617, 254, 255, 256, 257, + 618, 258, 259, 260, 261, 262, 263, 264, 619, 265, + 620, 621, 268, 269, 270, 271, 272, 622, 623, 624, + 625, 626, 276, 627, 628, 279, 629, 281, 282, 283, + 284, 285, 286, 630, 631, 287, 632, 289, 633, 634, + 291, 292, 293, 294, 295, 296, 297, 298, 635, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 636, + 637, 638, 323, 324, 325, 639, 640, 327, 328, 641, + 330, 642, 643, 332, 644, 334, 335, 336, 645, 337, + 338, 646, 647, 339, 340, 341, 648, 649, 342, 343, + 650, 651, 346, 652, 653, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 654, 655, 656, 657, + 360, 361, 658, 659, 364, 365, 660, 367, 368, 369, + 661, 370, 371, 372, 373, 374, 375, 662, 376, 377, + 378, 663, 380, 381, 382, 383, 664, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 665, 397, 398, 666, 400, 401, 402, 667, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 668, 669, 417, 418, 419, 420, 421, 2229, 670, + 424, 425, 671, 672, 427, 428, 673, 430, 674, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 675, 444, 676, 677, 678, 446, 447, 679, 448, + 680, 450, 451, 452, 453, 454, 681, 455, 682, 683, + 684, 685, 458, 459, 686, 461, 687, 688, 463, 464, + 689, 466, 467, 468, 469, 470, 690, 691, 471, 472, + 473, 692, 474, 475, 476, 477, 693, 478, 479, 480, + 481, 482, 694, 695, 696, 485, 697, 487, 488, 489, + 490, 491, 492, 493, 698, 699, 494, 700, 701, 495, + 496, 497, 498, 499, 500, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 512, 513, 514, 515, + 964, 0, 815, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 560, 0, 0, 0, 0, 565, + 129, 130, 0, 131, 132, 133, 567, 135, 136, 137, + 568, 569, 570, 571, 572, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 576, 577, + 155, 0, 156, 157, 158, 159, 579, 0, 581, 0, + 583, 163, 164, 165, 166, 167, 584, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 587, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 589, + 190, 191, 590, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 600, + 221, 222, 223, 224, 225, 601, 1340, 227, 0, 228, + 229, 604, 231, 0, 232, 0, 233, 607, 0, 609, + 236, 237, 610, 611, 240, 0, 241, 0, 614, 615, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 617, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 620, 621, 268, 269, 270, + 271, 272, 622, 623, 0, 625, 0, 276, 627, 628, + 279, 629, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 632, 289, 633, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 635, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 636, 637, 638, 323, 324, 325, + 639, 0, 327, 328, 641, 330, 0, 643, 332, 644, + 334, 335, 336, 0, 337, 338, 1341, 0, 339, 340, + 341, 0, 0, 342, 343, 650, 651, 346, 652, 653, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 658, 659, 364, + 365, 660, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 663, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 666, 400, + 401, 402, 667, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 669, 417, 418, + 419, 420, 421, 422, 670, 424, 425, 0, 672, 427, + 428, 673, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 675, 444, 676, 0, + 0, 446, 447, 0, 448, 680, 450, 451, 452, 453, + 454, 0, 455, 682, 683, 0, 0, 458, 459, 686, + 461, 687, 1342, 463, 464, 689, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 694, 695, 0, + 485, 697, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 702, 703, 704, 705, 706, 707, 708, 709, 710, 711, + 712, 512, 513, 514, 515, 964, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 3, 4, 0, - 776, 0, 0, 0, 0, 777, 123, 124, 0, 125, - 126, 127, 778, 129, 130, 131, 779, 780, 781, 782, - 783, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 784, 785, 149, 0, 150, 151, - 152, 153, 786, 0, 787, 0, 788, 157, 158, 159, - 160, 161, 789, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 790, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 791, 184, 185, 792, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 793, 216, 217, 218, 219, - 220, 794, 0, 222, 0, 223, 224, 795, 226, 0, - 227, 0, 228, 796, 0, 797, 231, 232, 798, 799, - 235, 0, 236, 0, 800, 801, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 802, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 803, 804, 263, 264, 265, 266, 267, 805, 806, - 0, 807, 0, 271, 808, 809, 274, 810, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 811, 284, 812, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 813, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 814, 815, 816, 318, 319, 320, 817, 0, 322, 323, - 818, 325, 0, 819, 327, 820, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 821, 822, 341, 823, 824, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 825, 826, 358, 359, 827, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 828, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 829, 394, 395, 396, 830, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 831, 409, 410, 411, 412, 413, 414, 832, 416, 417, - 418, 833, 420, 421, 834, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 835, - 437, 836, 439, 0, 440, 441, 0, 442, 837, 444, - 445, 446, 447, 448, 0, 449, 838, 839, 0, 0, - 452, 453, 840, 455, 841, 0, 457, 458, 842, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 1195, 844, 0, 479, 845, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 846, 847, 848, 849, 850, 851, 852, - 853, 854, 855, 856, 506, 507, 508, 509, 110, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 3, 4, 0, 560, + 0, 0, 0, 0, 565, 129, 130, 0, 131, 132, + 133, 567, 135, 136, 137, 568, 569, 570, 571, 572, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 576, 577, 155, 0, 156, 157, 158, + 159, 579, 0, 581, 0, 583, 163, 164, 165, 166, + 167, 584, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 587, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 589, 190, 191, 590, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 600, 221, 222, 223, 224, 225, + 601, 0, 227, 0, 228, 229, 604, 231, 0, 232, + 0, 233, 607, 0, 609, 236, 237, 610, 611, 240, + 0, 241, 0, 614, 615, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 617, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 620, 621, 268, 269, 270, 271, 272, 622, 623, 0, + 625, 0, 276, 627, 628, 279, 629, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 632, 289, 633, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 635, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 636, + 637, 638, 323, 324, 325, 639, 0, 327, 328, 641, + 330, 0, 643, 332, 644, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 650, 651, 346, 652, 653, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 658, 659, 364, 365, 660, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 663, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 666, 400, 401, 402, 667, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 669, 417, 418, 419, 420, 421, 422, 670, + 424, 425, 0, 672, 427, 428, 673, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 675, 444, 676, 0, 0, 446, 447, 0, 448, + 680, 450, 451, 452, 453, 454, 0, 455, 682, 683, + 0, 0, 458, 459, 686, 461, 687, 0, 463, 464, + 689, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 694, 695, 0, 485, 697, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 702, 703, 704, 705, 706, + 707, 708, 709, 710, 711, 712, 512, 513, 514, 515, + 116, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 128, + 129, 130, 0, 131, 132, 133, 134, 135, 136, 137, + 138, 139, 140, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 785, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 786, 0, 787, 0, + 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 178, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 788, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 234, 0, 235, + 236, 237, 238, 239, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 253, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 266, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 277, 278, + 279, 280, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 288, 289, 290, 0, 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, 0, 327, 328, 329, 330, 0, 790, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 345, 346, 347, 792, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 793, 363, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 426, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 443, 444, 795, 0, + 0, 446, 447, 0, 448, 449, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 796, + 461, 797, 0, 463, 464, 798, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 116, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 122, 123, 124, - 0, 125, 126, 127, 128, 129, 130, 131, 132, 133, - 134, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 612, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 613, 0, 614, 0, 156, 157, - 158, 159, 160, 161, 162, 163, 164, 165, 0, 166, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 128, 129, 130, 0, 131, 132, + 133, 134, 135, 136, 137, 138, 139, 140, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 161, 0, 162, 163, 164, 165, 166, 167, 168, 169, 170, 171, 0, 172, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 615, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 229, 0, 230, 231, 232, - 233, 234, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 248, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 261, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 272, 273, 274, 275, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 283, - 284, 285, 0, 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, 0, - 322, 323, 324, 325, 0, 617, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 340, 341, 342, 619, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 620, 357, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 393, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 419, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 622, 439, 0, 440, 441, 0, 442, - 443, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 623, 455, 624, 0, 457, 458, - 625, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 110, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 122, - 123, 124, 0, 125, 126, 127, 128, 129, 130, 131, - 132, 133, 134, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 155, 0, - 156, 157, 158, 159, 160, 161, 162, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 172, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 229, 0, 230, - 231, 232, 233, 234, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 248, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 261, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 272, 273, - 274, 275, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 283, 284, 285, 0, 286, 287, 288, 289, 290, + 176, 177, 0, 178, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 234, 0, 235, 236, 237, 238, 239, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 253, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 266, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 277, 278, 279, 280, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 288, 289, 290, 0, 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, 0, 322, 323, 324, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 340, 341, 342, 343, + 321, 322, 323, 324, 325, 326, 0, 327, 328, 329, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 357, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 393, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 438, 439, 0, 440, 441, - 0, 442, 443, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 1694, 150, 151, 152, 153, 154, 0, - 0, 1695, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 1696, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 1697, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 1698, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 1699, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 1700, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 426, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, 444, 445, 0, 0, 446, 447, 0, 448, + 449, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 0, 0, 0, 0, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 1735, 156, 157, 158, 159, 160, 0, 0, 1736, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 1737, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 1738, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 1739, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 1740, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 1741, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 1694, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 1696, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 1697, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 2254, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 1699, 364, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 1735, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 1737, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 1738, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 2309, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 1740, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 1741, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 3, 4, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 450, 451, 0, 0, 452, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 1700, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 0, 540, 0, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 3, 4, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 541, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 542, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 540, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 550, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 618, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 540, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 722, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 618, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 551, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 540, 0, 0, 0, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 791, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 752, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 896, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 791, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 938, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 450, 451, 0, 0, 452, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 1889, 0, 0, - 0, 0, 1890, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 1897, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 1898, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 1899, 442, - 0, 444, 1900, 446, 1901, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 1902, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 540, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 1939, 0, 0, 0, + 0, 1940, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 642, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 649, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 650, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 651, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 1947, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 1948, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 652, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 653, 451, 0, 0, 654, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 0, 540, 0, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 1949, 448, 0, 450, 1950, 452, 1951, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 1952, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 686, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 717, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 540, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 720, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 540, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 724, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 540, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 741, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 815, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 450, 451, 0, 0, 452, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 0, 540, 0, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 767, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 770, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 891, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 246, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 822, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 823, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 824, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 914, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 825, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 826, 457, + 0, 0, 827, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 649, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 653, 451, 0, 0, 654, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 517, 0, 540, 0, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 859, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 123, 124, 0, 125, - 126, 127, 0, 129, 130, 131, 132, 133, 0, 135, - 136, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 147, 148, 149, 0, 150, 151, - 152, 153, 154, 0, 0, 0, 156, 157, 158, 159, - 160, 161, 0, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 0, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 183, 184, 185, 186, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 202, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 215, 216, 217, 218, 219, - 220, 221, 0, 222, 0, 223, 224, 225, 226, 0, - 227, 0, 228, 0, 0, 0, 231, 232, 518, 0, - 235, 0, 236, 0, 237, 238, 239, 240, 0, 241, - 242, 243, 244, 245, 1261, 247, 0, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 0, 262, 263, 264, 265, 266, 267, 268, 269, - 0, 270, 0, 271, 0, 0, 274, 0, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 0, 284, 0, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 519, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 315, 0, 317, 318, 319, 320, 321, 0, 322, 323, - 0, 325, 0, 326, 327, 328, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 339, 0, 341, 0, 343, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 356, 0, 358, 359, 360, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 373, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 0, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 0, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 0, 420, 421, 422, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 520, - 437, 438, 439, 0, 440, 441, 0, 442, 0, 444, - 445, 446, 447, 448, 0, 449, 450, 451, 0, 0, - 452, 453, 454, 455, 456, 0, 457, 458, 459, 460, - 461, 462, 463, 464, 0, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 477, 478, 0, 479, 0, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 495, 496, 497, 498, 499, 500, 501, - 502, 503, 504, 505, 506, 507, 508, 509, 517, 0, - 540, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 123, 124, - 0, 125, 126, 127, 0, 129, 130, 131, 132, 133, - 0, 135, 136, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 147, 148, 149, 0, - 150, 151, 152, 153, 154, 0, 0, 0, 156, 157, - 158, 159, 160, 161, 0, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 0, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 183, 184, 185, - 186, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 202, 203, 204, 0, 205, 206, 207, 0, 208, - 209, 210, 0, 211, 212, 213, 214, 215, 216, 217, - 218, 219, 220, 221, 0, 222, 0, 223, 224, 225, - 226, 0, 227, 0, 228, 0, 0, 0, 231, 232, - 518, 0, 235, 0, 236, 0, 237, 238, 239, 240, - 0, 241, 242, 243, 244, 245, 1263, 247, 0, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 0, 262, 263, 264, 265, 266, 267, - 268, 269, 0, 270, 0, 271, 0, 0, 274, 0, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 0, - 284, 0, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 519, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 315, 0, 317, 318, 319, 320, 321, 0, - 322, 323, 0, 325, 0, 326, 327, 328, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 339, 0, 341, 0, 343, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 356, 0, 358, 359, 360, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 373, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 0, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 0, 409, 410, 411, 412, 413, 414, 415, - 416, 417, 418, 0, 420, 421, 422, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 520, 437, 438, 439, 0, 440, 441, 0, 442, - 0, 444, 445, 446, 447, 448, 0, 449, 450, 451, - 0, 0, 452, 453, 454, 455, 456, 0, 457, 458, - 459, 460, 461, 462, 463, 464, 0, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 477, 478, 0, 479, 0, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 495, 496, 497, 498, 499, - 500, 501, 502, 503, 504, 505, 506, 507, 508, 509, - 517, 0, 540, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 111, - 112, 113, 114, 115, 116, 117, 118, 0, 119, 120, - 121, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 123, 124, 0, 125, 126, 127, 0, 129, 130, 131, - 132, 133, 0, 135, 136, 0, 137, 138, 139, 140, - 141, 142, 0, 0, 143, 144, 145, 146, 147, 148, - 149, 0, 150, 151, 152, 153, 154, 0, 0, 0, - 156, 157, 158, 159, 160, 161, 0, 163, 164, 165, - 0, 166, 167, 168, 169, 170, 171, 0, 0, 173, - 174, 175, 176, 177, 178, 179, 180, 181, 182, 183, - 184, 185, 186, 187, 0, 188, 0, 189, 190, 191, - 192, 193, 194, 0, 195, 196, 197, 198, 199, 0, - 0, 200, 201, 202, 203, 204, 0, 205, 206, 207, - 0, 208, 209, 210, 0, 211, 212, 213, 214, 215, - 216, 217, 218, 219, 220, 221, 0, 222, 0, 223, - 224, 225, 226, 0, 227, 0, 228, 0, 0, 0, - 231, 232, 518, 0, 235, 0, 236, 0, 237, 238, - 239, 240, 0, 241, 242, 243, 244, 245, 1266, 247, - 0, 249, 250, 251, 252, 0, 253, 254, 255, 256, - 257, 258, 259, 0, 260, 0, 262, 263, 264, 265, - 266, 267, 268, 269, 0, 270, 0, 271, 0, 0, - 274, 0, 276, 277, 278, 279, 280, 281, 0, 0, - 282, 0, 284, 0, 0, 286, 287, 288, 289, 290, - 291, 292, 293, 519, 295, 296, 297, 298, 299, 300, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 891, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, 0, 317, 318, 319, 320, - 321, 0, 322, 323, 0, 325, 0, 326, 327, 328, - 329, 330, 331, 0, 332, 333, 0, 0, 334, 335, - 336, 0, 0, 337, 338, 339, 0, 341, 0, 343, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 0, 0, 0, 0, 354, 355, 356, 0, 358, 359, - 360, 361, 362, 363, 0, 364, 365, 366, 367, 368, - 369, 0, 370, 371, 372, 373, 374, 375, 376, 377, - 0, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 0, 391, 392, 0, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 0, 0, 409, 410, 411, 412, 413, - 414, 415, 416, 417, 418, 0, 420, 421, 422, 423, - 0, 424, 425, 426, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 520, 437, 438, 439, 0, 440, 441, - 0, 442, 0, 444, 445, 446, 447, 448, 0, 449, - 450, 451, 0, 0, 452, 453, 454, 455, 456, 0, - 457, 458, 459, 460, 461, 462, 463, 464, 0, 0, - 465, 466, 467, 0, 468, 469, 470, 471, 0, 472, - 473, 474, 475, 476, 477, 478, 0, 479, 0, 481, - 482, 483, 484, 485, 486, 487, 0, 0, 488, 0, - 0, 489, 490, 491, 492, 493, 494, 495, 496, 497, - 498, 499, 500, 501, 502, 503, 504, 505, 506, 507, - 508, 509, 517, 0, 540, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 239, 240, 0, 241, 242, 243, 244, 245, - 1268, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 371, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 517, 0, 540, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 123, 124, 0, 125, 126, 127, - 0, 129, 130, 131, 132, 133, 0, 135, 136, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 147, 148, 149, 0, 150, 151, 152, 153, - 154, 0, 0, 0, 156, 157, 158, 159, 160, 161, - 0, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 0, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 183, 184, 185, 186, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 202, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 215, 216, 217, 218, 219, 220, 221, - 0, 222, 0, 223, 224, 225, 226, 0, 227, 0, - 228, 0, 0, 0, 231, 232, 518, 0, 235, 0, - 236, 0, 237, 238, 239, 240, 0, 241, 242, 243, - 244, 245, 2174, 247, 0, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 0, - 262, 263, 264, 265, 266, 267, 268, 269, 0, 270, - 0, 271, 0, 0, 274, 0, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 0, 284, 0, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 519, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 315, 0, - 317, 318, 319, 320, 321, 0, 322, 323, 0, 325, - 0, 326, 327, 328, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 339, - 0, 341, 0, 343, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 356, 0, 358, 359, 360, 361, 362, 363, 0, 364, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 894, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 0, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 0, 409, - 410, 411, 412, 413, 414, 415, 416, 417, 418, 0, - 420, 421, 422, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 520, 437, 438, - 439, 0, 440, 441, 0, 442, 0, 444, 445, 446, - 447, 448, 0, 449, 450, 451, 0, 0, 452, 453, - 454, 455, 456, 0, 457, 458, 459, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 477, 478, - 0, 479, 0, 481, 482, 483, 484, 485, 486, 487, - 0, 0, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 495, 496, 497, 498, 499, 500, 501, 502, 503, - 504, 505, 506, 507, 508, 509, 1431, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 111, 112, 113, 114, 115, - 116, 117, 118, 0, 119, 120, 121, 0, 0, 0, - 1432, 0, 0, -765, 0, 1433, 123, 124, 0, 125, - 126, 127, 1434, 129, 130, 131, 0, 1435, 1436, 1437, - 1438, 0, 137, 138, 139, 140, 141, 142, 0, 0, - 143, 144, 145, 146, 1439, 1440, 149, 0, 150, 151, - 152, 153, 0, 0, 1441, 0, 1442, 157, 158, 159, - 160, 161, 1443, 163, 164, 165, 0, 166, 167, 168, - 169, 170, 171, 0, 1444, 173, 174, 175, 176, 177, - 178, 179, 180, 181, 182, 1445, 184, 185, 1446, 187, - 0, 188, 0, 189, 190, 191, 192, 193, 194, 0, - 195, 196, 197, 198, 199, 0, 0, 200, 201, 950, - 203, 204, 0, 205, 206, 207, 0, 208, 209, 210, - 0, 211, 212, 213, 214, 0, 216, 217, 218, 219, - 220, 0, 0, 222, 0, 223, 224, 1447, 226, 0, - 227, 0, 228, 1448, 0, 1449, 231, 232, -765, 1450, - 235, 0, 236, 0, 0, 0, 239, 240, 0, 241, - 242, 243, 244, 245, 246, 247, 1451, 249, 250, 251, - 252, 0, 253, 254, 255, 256, 257, 258, 259, 0, - 260, 1452, 0, 263, 264, 265, 266, 267, 1453, 1454, - 0, 1455, 0, 271, 1456, 1457, 274, 1458, 276, 277, - 278, 279, 280, 281, 0, 0, 282, 1459, 284, 1460, - 0, 286, 287, 288, 289, 290, 291, 292, 293, 1461, - 295, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, - 1462, 1463, 1464, 318, 319, 320, 0, 0, 322, 323, - 1465, 325, 0, 0, 327, 1466, 329, 330, 331, 0, - 332, 333, 0, 0, 334, 335, 336, 0, 0, 337, - 338, 0, 1467, 341, 1468, 0, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 0, 0, 0, 0, - 354, 355, 0, 1469, 358, 359, 0, 361, 362, 363, - 0, 364, 365, 366, 367, 368, 369, 0, 370, 371, - 372, 1470, 374, 375, 376, 377, 0, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 0, 391, 392, 1471, 394, 395, 396, 1472, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 0, - 1473, 409, 410, 411, 412, 413, 414, 1474, 416, 417, - 418, 1475, 420, 421, 1476, 423, 0, 424, 425, 426, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 1477, - 437, 0, 439, 0, 440, 441, 0, 442, 1478, 444, - 445, 446, 447, 448, 0, 449, 1479, 1480, 0, 0, - 452, 453, 0, 455, 0, 0, 457, 458, 1481, 460, - 461, 462, 463, 464, 1482, 0, 465, 466, 467, 0, - 468, 469, 470, 471, 0, 472, 473, 474, 475, 476, - 0, 1483, 0, 479, 1484, 481, 482, 483, 484, 485, - 486, 487, 0, 0, 488, 0, 0, 489, 490, 491, - 492, 493, 494, 517, 0, 540, 0, 0, 0, 0, - 0, 0, 0, 0, 506, 507, 508, 509, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 123, 124, 0, 125, 126, 127, 0, - 129, 130, 131, 132, 133, 0, 135, 136, 0, 137, - 138, 139, 140, 141, 142, 0, 0, 143, 144, 145, - 146, 147, 148, 149, 0, 150, 151, 152, 153, 154, - 0, 0, 0, 156, 157, 158, 159, 160, 161, 0, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 0, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 183, 184, 185, 186, 187, 0, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 202, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 215, 216, 217, 218, 219, 220, 221, 0, - 222, 0, 223, 224, 225, 226, 0, 227, 0, 228, - 0, 0, 0, 231, 232, 518, 0, 235, 0, 236, - 0, 237, 238, 239, 240, 0, 241, 242, 243, 244, - 245, 2899, 247, 0, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 0, 262, - 263, 264, 265, 266, 267, 268, 269, 0, 270, 0, - 271, 0, 0, 274, 0, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 0, 284, 0, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 519, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 315, 0, 317, - 318, 319, 320, 321, 0, 322, 323, 0, 325, 0, - 326, 327, 328, 329, 330, 331, 0, 332, 333, 0, - 0, 334, 335, 336, 0, 0, 337, 338, 339, 0, - 341, 0, 343, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 356, - 0, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 373, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 0, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 0, 409, 410, - 411, 412, 413, 414, 415, 416, 417, 418, 0, 420, - 421, 422, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 520, 437, 438, 439, - 0, 440, 441, 0, 442, 0, 444, 445, 446, 447, - 448, 0, 449, 450, 451, 0, 0, 452, 453, 454, - 455, 456, 0, 457, 458, 459, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 478, 0, - 479, 0, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 517, 0, 0, 0, 0, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 123, 124, 0, 125, 126, - 127, 0, 129, 130, 131, 132, 133, 0, 135, 136, - 0, 137, 138, 139, 140, 141, 142, 0, 0, 143, - 144, 145, 146, 147, 148, 149, 0, 150, 151, 152, - 153, 154, 0, 0, 0, 156, 157, 158, 159, 160, - 161, 0, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 0, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 183, 184, 185, 186, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 202, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, - 221, 0, 222, 0, 223, 224, 225, 226, 0, 227, - 0, 228, 0, 0, 0, 231, 232, 518, 0, 235, - 0, 236, 0, 237, 238, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 0, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 0, 262, 263, 264, 265, 266, 267, 268, 269, 0, - 270, 0, 271, 0, 0, 274, 0, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 0, 284, 0, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 519, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 898, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 926, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, - 0, 317, 318, 319, 320, 321, 0, 322, 323, 0, - 325, 0, 326, 327, 328, 329, 330, 331, 0, 332, - 333, 0, 0, 334, 335, 336, 0, 0, 337, 338, - 339, 0, 341, 0, 343, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 356, 0, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 373, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 0, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 0, 420, 421, 422, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 520, 437, - 438, 439, 0, 440, 441, 0, 442, 0, 444, 445, - 446, 447, 448, 0, 449, 450, 451, 0, 0, 452, - 453, 454, 455, 456, 0, 457, 458, 459, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 478, 0, 479, 0, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 517, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 123, 124, 0, - 125, 126, 127, 0, 129, 130, 131, 132, 133, 0, - 135, 136, 0, 137, 138, 139, 140, 141, 142, 0, - 0, 143, 144, 145, 146, 147, 148, 149, 0, 150, - 151, 152, 153, 154, 0, 0, 0, 156, 157, 158, - 159, 160, 161, 0, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 0, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 202, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 221, 0, 222, 0, 223, 224, 225, 226, - 0, 227, 0, 228, 0, 0, 0, 231, 232, 518, - 0, 664, 0, 236, 0, 237, 238, 239, 240, 0, - 241, 242, 243, 244, 245, 246, 247, 0, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 0, 262, 263, 264, 265, 266, 267, 268, - 269, 0, 270, 0, 271, 0, 0, 274, 0, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 0, 284, - 0, 0, 286, 287, 665, 289, 290, 291, 292, 293, - 519, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 0, 317, 318, 319, 320, 321, 0, 322, - 323, 0, 325, 0, 326, 327, 328, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 0, 341, 0, 343, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 356, 0, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 373, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 0, 394, 395, 396, 397, 398, - 399, 400, 401, 666, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 667, 414, 415, 416, - 417, 418, 0, 420, 421, 422, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 520, 437, 438, 439, 0, 440, 441, 0, 442, 0, - 444, 445, 446, 447, 448, 0, 449, 668, 451, 0, - 0, 452, 453, 454, 455, 456, 0, 457, 458, 459, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 478, 0, 479, 0, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 517, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 123, - 124, 0, 125, 126, 127, 0, 129, 130, 131, 132, - 133, 0, 135, 136, 0, 137, 138, 139, 140, 141, - 142, 0, 0, 143, 144, 145, 146, 147, 148, 149, - 0, 150, 151, 152, 153, 154, 0, 0, 0, 156, - 157, 158, 159, 160, 161, 0, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 0, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 202, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 215, 216, - 217, 218, 219, 220, 221, 0, 222, 0, 223, 224, - 225, 226, 0, 227, 0, 228, 0, 0, 0, 231, - 232, 518, 0, 235, 0, 236, 0, 237, 238, 239, - 240, 0, 241, 242, 243, 244, 245, 763, 247, 0, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 0, 262, 263, 264, 265, 266, - 267, 268, 269, 0, 270, 0, 271, 0, 0, 274, - 0, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 0, 284, 0, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 519, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 315, 0, 317, 318, 319, 320, 321, - 0, 322, 323, 0, 325, 0, 326, 327, 328, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 0, 341, 0, 343, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 356, 0, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 373, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 0, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 0, 420, 421, 422, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 520, 437, 438, 439, 0, 440, 441, 0, - 442, 0, 444, 445, 446, 447, 448, 0, 449, 450, - 451, 0, 0, 452, 453, 454, 455, 456, 0, 457, - 458, 459, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 478, 0, 479, 0, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 517, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 954, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 957, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 123, 124, 0, 125, 126, 127, 0, 129, 130, - 131, 132, 133, 0, 135, 136, 0, 137, 138, 139, - 140, 141, 142, 0, 0, 143, 144, 145, 146, 147, - 148, 149, 0, 150, 151, 152, 153, 154, 0, 0, - 0, 156, 157, 158, 159, 160, 161, 0, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 0, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 183, 184, 185, 186, 187, 0, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 202, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 215, 216, 217, 218, 219, 220, 221, 0, 222, 0, - 223, 224, 225, 226, 0, 227, 0, 228, 0, 0, - 0, 231, 232, 518, 0, 235, 0, 236, 0, 237, - 238, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 0, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 0, 262, 263, 264, - 265, 266, 267, 268, 269, 0, 270, 0, 271, 0, - 0, 274, 0, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 0, 284, 0, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 519, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 315, 0, 317, 318, 319, - 320, 321, 0, 322, 323, 0, 325, 0, 326, 327, - 328, 329, 330, 331, 0, 332, 333, 0, 0, 334, - 335, 336, 0, 0, 337, 338, 339, 0, 341, 0, - 343, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 0, 0, 0, 0, 354, 355, 356, 0, 358, - 359, 360, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 373, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 0, 394, - 395, 396, 397, 398, 399, 400, 401, 666, 403, 404, - 405, 406, 407, 408, 0, 0, 409, 410, 411, 412, - 413, 414, 415, 416, 417, 418, 0, 420, 421, 422, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 520, 437, 438, 439, 0, 440, - 441, 0, 442, 0, 444, 445, 446, 447, 448, 0, - 449, 668, 451, 0, 0, 452, 453, 454, 455, 456, - 0, 457, 458, 459, 460, 461, 462, 463, 464, 0, - 0, 465, 466, 467, 0, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 477, 478, 0, 479, 0, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 0, 489, 490, 491, 492, 493, 494, 495, 496, - 497, 498, 499, 500, 501, 502, 503, 504, 505, 506, - 507, 508, 509, 517, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 1000, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 1023, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 111, 112, 113, 114, 115, 116, 117, 118, - 0, 119, 120, 121, 0, 0, 0, 0, 0, 0, - 1626, 0, 0, 123, 124, 0, 125, 126, 127, 0, - 129, 130, 131, 132, 133, 0, 135, 136, 0, 137, - 138, 139, 140, 141, 142, 0, 0, 143, 144, 145, - 146, 147, 148, 149, 0, 150, 151, 152, 153, 154, - 0, 0, 0, 156, 157, 158, 159, 160, 161, 0, - 163, 164, 165, 0, 166, 167, 168, 169, 170, 171, - 0, 0, 173, 174, 175, 176, 177, 178, 179, 180, - 181, 182, 183, 184, 185, 186, 187, 0, 188, 0, - 189, 190, 191, 192, 193, 194, 0, 195, 196, 197, - 198, 199, 0, 0, 200, 201, 202, 203, 204, 0, - 205, 206, 207, 0, 208, 209, 210, 0, 211, 212, - 213, 214, 215, 216, 217, 218, 219, 220, 221, 0, - 222, 0, 223, 224, 225, 226, 0, 227, 0, 228, - 0, 0, 0, 231, 232, 518, 0, 235, 0, 236, - 0, 237, 238, 239, 240, 0, 241, 242, 243, 244, - 245, 246, 247, 0, 249, 250, 251, 252, 0, 253, - 254, 255, 256, 257, 258, 259, 0, 260, 0, 262, - 263, 264, 265, 266, 267, 268, 269, 0, 270, 0, - 271, 0, 0, 274, 0, 276, 277, 278, 279, 280, - 281, 0, 0, 282, 0, 284, 0, 0, 286, 287, - 288, 289, 290, 291, 292, 293, 519, 295, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 315, 0, 317, - 318, 319, 320, 321, 0, 322, 323, 0, 325, 0, - 326, 327, 328, 329, 330, 331, 0, 332, 333, 0, - 0, 334, 335, 336, 0, 0, 337, 338, 339, 0, - 341, 0, 343, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 0, 0, 0, 0, 354, 355, 356, - 0, 358, 359, 360, 361, 362, 363, 0, 364, 365, - 366, 367, 368, 369, 0, 370, 371, 372, 373, 374, - 375, 376, 377, 0, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 0, 391, 392, - 0, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 0, 0, 409, 410, - 411, 412, 413, 0, 415, 416, 417, 418, 0, 420, - 421, 422, 423, 0, 424, 425, 426, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 520, 437, 438, 439, - 0, 440, 441, 0, 442, 0, 444, 445, 446, 447, - 448, 0, 449, 450, 451, 0, 0, 452, 453, 454, - 455, 456, 0, 457, 458, 459, 460, 461, 462, 463, - 464, 0, 0, 465, 466, 467, 0, 468, 469, 470, - 471, 0, 472, 473, 474, 475, 476, 477, 478, 0, - 479, 0, 481, 482, 483, 484, 485, 486, 487, 0, - 0, 488, 0, 0, 489, 490, 491, 492, 493, 494, - 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, - 505, 506, 507, 508, 509, 517, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 822, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 826, 457, + 0, 0, 827, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 1298, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 123, 124, 0, 125, 126, - 127, 0, 129, 130, 131, 132, 133, 0, 135, 136, - 0, 137, 138, 139, 140, 141, 142, 0, 0, 143, - 144, 145, 146, 147, 148, 149, 0, 150, 151, 152, - 153, 154, 0, 0, 0, 156, 157, 158, 159, 160, - 161, 0, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 0, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 183, 184, 185, 186, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 202, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 215, 216, 217, 218, 219, 220, - 221, 0, 222, 0, 223, 224, 225, 226, 0, 227, - 0, 228, 0, 0, 0, 231, 232, 518, 0, 235, - 0, 236, 0, 237, 238, 239, 240, 0, 241, 242, - 243, 244, 245, 1796, 247, 0, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 0, 262, 263, 264, 265, 266, 267, 268, 269, 0, - 270, 0, 271, 0, 0, 274, 0, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 0, 284, 0, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 519, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 1300, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 1303, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, - 0, 317, 318, 319, 320, 321, 0, 322, 323, 0, - 325, 0, 326, 327, 328, 329, 330, 331, 0, 332, - 333, 0, 0, 334, 335, 336, 0, 0, 337, 338, - 339, 0, 341, 0, 343, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 356, 0, 358, 359, 360, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 373, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 0, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 0, - 409, 410, 411, 412, 413, 414, 415, 416, 417, 418, - 0, 420, 421, 422, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 520, 437, - 438, 439, 0, 440, 441, 0, 442, 0, 444, 445, - 446, 447, 448, 0, 449, 450, 451, 0, 0, 452, - 453, 454, 455, 456, 0, 457, 458, 459, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 477, - 478, 0, 479, 0, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 495, 496, 497, 498, 499, 500, 501, 502, - 503, 504, 505, 506, 507, 508, 509, 517, 0, 0, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 111, 112, 113, 114, - 115, 116, 117, 118, 0, 119, 120, 121, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 123, 124, 0, - 125, 126, 127, 0, 129, 130, 131, 132, 133, 0, - 135, 136, 0, 137, 138, 139, 140, 141, 142, 0, - 0, 143, 144, 145, 146, 147, 148, 149, 0, 150, - 151, 152, 153, 154, 0, 0, 0, 156, 157, 158, - 159, 160, 161, 0, 163, 164, 165, 0, 166, 167, - 168, 169, 170, 171, 0, 0, 173, 174, 175, 176, - 177, 178, 179, 180, 181, 182, 183, 184, 185, 186, - 187, 0, 188, 0, 189, 190, 191, 192, 193, 194, - 0, 195, 196, 197, 198, 199, 0, 0, 200, 201, - 202, 203, 204, 0, 205, 206, 207, 0, 208, 209, - 210, 0, 211, 212, 213, 214, 215, 216, 217, 218, - 219, 220, 221, 0, 222, 0, 223, 224, 225, 226, - 0, 227, 0, 228, 0, 0, 0, 231, 232, 518, - 0, 235, 0, 236, 0, 237, 238, 239, 240, 0, - 241, 242, 243, 244, 245, 2161, 247, 0, 249, 250, - 251, 252, 0, 253, 254, 255, 256, 257, 258, 259, - 0, 260, 0, 262, 263, 264, 265, 266, 267, 268, - 269, 0, 270, 0, 271, 0, 0, 274, 0, 276, - 277, 278, 279, 280, 281, 0, 0, 282, 0, 284, - 0, 0, 286, 287, 288, 289, 290, 291, 292, 293, - 519, 295, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, 0, 317, 318, 319, 320, 321, 0, 322, - 323, 0, 325, 0, 326, 327, 328, 329, 330, 331, - 0, 332, 333, 0, 0, 334, 335, 336, 0, 0, - 337, 338, 339, 0, 341, 0, 343, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 0, 0, 0, - 0, 354, 355, 356, 0, 358, 359, 360, 361, 362, - 363, 0, 364, 365, 366, 367, 368, 369, 0, 370, - 371, 372, 373, 374, 375, 376, 377, 0, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 0, 391, 392, 0, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 0, 0, 409, 410, 411, 412, 413, 414, 415, 416, - 417, 418, 0, 420, 421, 422, 423, 0, 424, 425, - 426, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 520, 437, 438, 439, 0, 440, 441, 0, 442, 0, - 444, 445, 446, 447, 448, 0, 449, 450, 451, 0, - 0, 452, 453, 454, 455, 456, 0, 457, 458, 459, - 460, 461, 462, 463, 464, 0, 0, 465, 466, 467, - 0, 468, 469, 470, 471, 0, 472, 473, 474, 475, - 476, 477, 478, 0, 479, 0, 481, 482, 483, 484, - 485, 486, 487, 0, 0, 488, 0, 0, 489, 490, - 491, 492, 493, 494, 495, 496, 497, 498, 499, 500, - 501, 502, 503, 504, 505, 506, 507, 508, 509, 517, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 1305, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 549, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 2223, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 1472, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 111, 112, - 113, 114, 115, 116, 117, 118, 0, 119, 120, 121, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 123, - 124, 0, 125, 126, 127, 0, 129, 130, 131, 132, - 133, 0, 135, 136, 0, 137, 138, 139, 140, 141, - 142, 0, 0, 143, 144, 145, 146, 147, 148, 149, - 0, 150, 151, 152, 153, 154, 0, 0, 0, 156, - 157, 158, 159, 160, 161, 0, 163, 164, 165, 0, - 166, 167, 168, 169, 170, 171, 0, 0, 173, 174, - 175, 176, 177, 178, 179, 180, 181, 182, 183, 184, - 185, 186, 187, 0, 188, 0, 189, 190, 191, 192, - 193, 194, 0, 195, 196, 197, 198, 199, 0, 0, - 200, 201, 202, 203, 204, 0, 205, 206, 207, 0, - 208, 209, 210, 0, 211, 212, 213, 214, 215, 216, - 217, 218, 219, 220, 221, 0, 222, 0, 223, 224, - 225, 226, 0, 227, 0, 228, 0, 0, 0, 231, - 232, 518, 0, 235, 0, 236, 0, 237, 238, 239, - 240, 0, 241, 242, 243, 244, 245, 2176, 247, 0, - 249, 250, 251, 252, 0, 253, 254, 255, 256, 257, - 258, 259, 0, 260, 0, 262, 263, 264, 265, 266, - 267, 268, 269, 0, 270, 0, 271, 0, 0, 274, - 0, 276, 277, 278, 279, 280, 281, 0, 0, 282, - 0, 284, 0, 0, 286, 287, 288, 289, 290, 291, - 292, 293, 519, 295, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 315, 0, 317, 318, 319, 320, 321, - 0, 322, 323, 0, 325, 0, 326, 327, 328, 329, - 330, 331, 0, 332, 333, 0, 0, 334, 335, 336, - 0, 0, 337, 338, 339, 0, 341, 0, 343, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 0, - 0, 0, 0, 354, 355, 356, 0, 358, 359, 360, - 361, 362, 363, 0, 364, 365, 366, 367, 368, 369, - 0, 370, 371, 372, 373, 374, 375, 376, 377, 0, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 0, 391, 392, 0, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 0, 0, 409, 410, 411, 412, 413, 414, - 415, 416, 417, 418, 0, 420, 421, 422, 423, 0, - 424, 425, 426, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 520, 437, 438, 439, 0, 440, 441, 0, - 442, 0, 444, 445, 446, 447, 448, 0, 449, 450, - 451, 0, 0, 452, 453, 454, 455, 456, 0, 457, - 458, 459, 460, 461, 462, 463, 464, 0, 0, 465, - 466, 467, 0, 468, 469, 470, 471, 0, 472, 473, - 474, 475, 476, 477, 478, 0, 479, 0, 481, 482, - 483, 484, 485, 486, 487, 0, 0, 488, 0, 0, - 489, 490, 491, 492, 493, 494, 495, 496, 497, 498, - 499, 500, 501, 502, 503, 504, 505, 506, 507, 508, - 509, 1431, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 1473, + 0, 0, -785, 0, 1474, 129, 130, 0, 131, 132, + 133, 1475, 135, 136, 137, 0, 1476, 1477, 1478, 1479, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 1480, 1481, 155, 0, 156, 157, 158, + 159, 0, 0, 1482, 0, 1483, 163, 164, 165, 166, + 167, 1484, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 1485, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 1486, 190, 191, 1487, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 1059, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 0, 221, 222, 223, 224, 225, + 0, 0, 227, 0, 228, 229, 1488, 231, 0, 232, + 0, 233, 1489, 0, 1490, 236, 237, -785, 1491, 240, + 0, 241, 0, 0, 0, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 1492, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 1493, 0, 268, 269, 270, 271, 272, 1494, 1495, 0, + 1496, 0, 276, 1497, 1498, 279, 1499, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 1500, 289, 1501, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 1502, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 1503, + 1504, 1505, 323, 324, 325, 0, 0, 327, 328, 1506, + 330, 0, 0, 332, 1507, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 0, 1508, 346, 1509, 0, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 0, 1510, 364, 365, 0, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 1511, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 1512, 400, 401, 402, 1513, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 1514, 417, 418, 419, 420, 421, 422, 1515, + 424, 425, 0, 1516, 427, 428, 1517, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 1518, 444, 0, 0, 0, 446, 447, 0, 448, + 1519, 450, 451, 452, 453, 454, 0, 455, 1520, 1521, + 0, 0, 458, 459, 0, 461, 0, 0, 463, 464, + 1522, 466, 467, 468, 469, 470, 1523, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 0, 1524, 0, 485, 1525, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 523, 0, 549, 0, 0, + 0, 0, 0, 0, 0, 0, 512, 513, 514, 515, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 2964, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 111, 112, 113, 114, 115, 116, 117, 118, 0, 119, - 120, 121, 0, 0, 0, 1432, 0, 0, 0, 0, - 1433, 123, 124, 0, 125, 126, 127, 1434, 129, 130, - 131, 0, 1435, 1436, 1437, 1438, 0, 137, 138, 139, - 140, 141, 142, 0, 0, 143, 144, 145, 146, 1439, - 1440, 149, 0, 150, 151, 152, 153, 0, 0, 1441, - 0, 1442, 157, 158, 159, 160, 161, 1443, 163, 164, - 165, 0, 166, 167, 168, 169, 170, 171, 0, 1444, - 173, 174, 175, 176, 177, 178, 179, 180, 181, 182, - 1445, 184, 185, 1446, 187, 0, 188, 0, 189, 190, - 191, 192, 193, 194, 0, 195, 196, 197, 198, 199, - 0, 0, 200, 201, 950, 203, 204, 0, 205, 206, - 207, 0, 208, 209, 210, 0, 211, 212, 213, 214, - 0, 216, 217, 218, 219, 220, 0, 0, 222, 0, - 223, 224, 1447, 226, 0, 227, 0, 228, 1448, 0, - 1449, 231, 232, 0, 1450, 235, 0, 236, 0, 0, - 0, 239, 240, 0, 241, 242, 243, 244, 245, 246, - 247, 1451, 249, 250, 251, 252, 0, 253, 254, 255, - 256, 257, 258, 259, 0, 260, 1452, 0, 263, 264, - 265, 266, 267, 1453, 1454, 0, 1455, 0, 271, 1456, - 1457, 274, 1458, 276, 277, 278, 279, 280, 281, 0, - 0, 282, 1459, 284, 1460, 0, 286, 287, 288, 289, - 290, 291, 292, 293, 1461, 295, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 1462, 1463, 1464, 318, 319, - 320, 0, 0, 322, 323, 1465, 325, 0, 0, 327, - 1466, 329, 330, 331, 0, 332, 333, 0, 0, 334, - 335, 336, 0, 0, 337, 338, 0, 1467, 341, 1468, - 0, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 0, 0, 0, 0, 354, 355, 0, 1469, 358, - 359, 0, 361, 362, 363, 0, 364, 365, 366, 367, - 368, 369, 0, 370, 371, 372, 1470, 374, 375, 376, - 377, 0, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 0, 391, 392, 1471, 394, - 395, 396, 1472, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 0, 1473, 409, 410, 411, 412, - 413, 414, 1474, 416, 417, 418, 1475, 420, 421, 1476, - 423, 0, 424, 425, 426, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 1477, 437, 0, 439, 0, 440, - 441, 0, 442, 1478, 444, 445, 446, 447, 448, 0, - 449, 1479, 1480, 0, 0, 452, 453, 0, 455, 0, - 0, 457, 458, 1481, 460, 461, 462, 463, 464, 1482, - 0, 465, 466, 467, 0, 468, 469, 470, 471, 0, - 472, 473, 474, 475, 476, 0, 1483, 0, 479, 1484, - 481, 482, 483, 484, 485, 486, 487, 0, 0, 488, - 0, 0, 489, 490, 491, 492, 493, 494, 1431, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 506, - 507, 508, 509, 0, 0, 0, 0, 111, 112, 113, - 114, 115, 116, 117, 118, 0, 119, 120, 121, 0, - 0, 0, 1432, 0, 0, 0, 0, 1433, 123, 124, - 0, 125, 126, 127, 1434, 129, 130, 131, 0, 1435, - 1436, 1437, 1438, 0, 137, 138, 139, 140, 141, 142, - 0, 0, 143, 144, 145, 146, 1439, 1440, 149, 0, - 150, 151, 152, 153, 0, 0, 1441, 0, 1442, 157, - 158, 159, 160, 161, 1443, 163, 164, 165, 0, 166, - 167, 168, 169, 170, 171, 0, 1444, 173, 174, 175, - 176, 177, 178, 179, 180, 181, 182, 1445, 184, 185, - 1446, 187, 0, 188, 0, 189, 190, 191, 192, 193, - 194, 0, 195, 196, 197, 198, 199, 0, 0, 200, - 201, 950, 203, 204, 0, 205, 206, 207, 0, 1781, - 209, 210, 0, 211, 212, 213, 214, 0, 216, 217, - 218, 219, 220, 0, 0, 222, 0, 223, 224, 1447, - 226, 0, 227, 0, 228, 1448, 0, 1449, 231, 232, - 0, 1450, 235, 0, 236, 0, 0, 0, 239, 240, - 0, 241, 242, 243, 244, 245, 246, 247, 1451, 249, - 250, 251, 252, 0, 253, 254, 255, 256, 257, 258, - 259, 0, 260, 1452, 0, 263, 264, 265, 266, 267, - 1453, 1454, 0, 1455, 0, 271, 1456, 1457, 274, 1458, - 276, 277, 278, 279, 280, 281, 0, 0, 282, 1459, - 284, 1460, 0, 286, 287, 288, 289, 290, 291, 292, - 293, 1461, 295, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - 313, 314, 1462, 1463, 1464, 318, 319, 320, 0, 0, - 322, 323, 1465, 325, 0, 0, 327, 1466, 329, 330, - 331, 0, 332, 333, 0, 0, 334, 335, 336, 0, - 0, 337, 338, 0, 1467, 341, 1468, 0, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 0, 0, - 0, 0, 354, 355, 0, 1469, 358, 359, 0, 361, - 362, 363, 0, 364, 365, 366, 367, 368, 369, 0, - 370, 371, 372, 1470, 374, 375, 376, 377, 0, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 0, 391, 392, 1471, 394, 395, 396, 1472, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 0, 1473, 409, 410, 411, 412, 413, 414, 1474, - 416, 417, 418, 1475, 420, 421, 1476, 423, 0, 424, - 425, 426, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 1477, 437, 0, 439, 0, 440, 441, 0, 442, - 1478, 444, 445, 446, 447, 448, 0, 449, 1479, 1480, - 0, 0, 452, 453, 0, 455, 0, 0, 457, 458, - 1481, 460, 461, 462, 463, 464, 1482, 0, 465, 466, - 467, 0, 468, 469, 470, 471, 0, 472, 473, 474, - 475, 476, 0, 1483, 0, 479, 1484, 481, 482, 483, - 484, 485, 486, 487, 0, 0, 488, 0, 0, 489, - 490, 491, 492, 493, 494, 3068, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 506, 507, 508, 509, - 0, 0, 0, 0, 111, 112, 113, 114, 115, 116, - 117, 118, 0, 119, 120, 121, 0, 0, 0, 2823, - 0, 0, 0, 0, 2824, 123, 124, 0, 125, 126, - 127, 2825, 129, 130, 131, 0, 1435, 2826, 1437, 1438, - 0, 137, 138, 139, 140, 141, 142, 0, 0, 143, - 144, 145, 146, 1439, 1440, 149, 0, 150, 151, 152, - 153, 0, 0, 2827, 0, 2828, 157, 158, 159, 160, - 161, 2829, 163, 164, 165, 0, 166, 167, 168, 169, - 170, 171, 0, 2830, 173, 174, 175, 176, 177, 178, - 179, 180, 181, 182, 1445, 184, 185, 1446, 187, 0, - 188, 0, 189, 190, 191, 192, 193, 194, 0, 195, - 196, 197, 198, 199, 0, 0, 200, 201, 950, 203, - 204, 0, 205, 206, 207, 0, 208, 209, 210, 0, - 211, 212, 213, 214, 0, 216, 217, 218, 219, 220, - 0, 0, 222, 0, 223, 224, 1447, 226, 0, 227, - 0, 228, 2831, 0, 2832, 231, 232, 2833, 2834, 235, - 0, 236, 0, 0, 0, 239, 240, 0, 241, 242, - 243, 244, 245, 246, 247, 2835, 249, 250, 251, 252, - 0, 253, 254, 255, 256, 257, 258, 259, 0, 260, - 2836, 0, 263, 264, 265, 266, 267, 1453, 1454, 0, - 1455, 0, 271, 2837, 2838, 274, 2839, 276, 277, 278, - 279, 280, 281, 0, 0, 282, 2840, 284, 2841, 0, - 286, 287, 288, 289, 290, 291, 292, 293, 3069, 295, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 1462, - 2843, 1464, 318, 319, 320, 0, 0, 322, 323, 2845, - 325, 0, 0, 327, 1466, 329, 330, 331, 0, 332, - 333, 0, 0, 334, 335, 336, 0, 0, 337, 338, - 0, 2847, 341, 2848, 0, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 0, 0, 0, 0, 354, - 355, 0, 2849, 358, 359, 0, 361, 362, 363, 0, - 364, 365, 366, 367, 368, 369, 0, 370, 371, 372, - 1470, 374, 375, 376, 377, 0, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 0, - 391, 392, 2850, 394, 395, 396, 0, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 0, 2851, - 409, 410, 411, 412, 413, 414, 0, 416, 417, 418, - 2853, 420, 421, 1476, 423, 0, 424, 425, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 3070, 437, - 0, 439, 0, 440, 441, 0, 442, 2855, 444, 445, - 446, 447, 448, 0, 449, 1479, 1480, 0, 0, 452, - 453, 0, 455, 0, 0, 457, 458, 2856, 460, 461, - 462, 463, 464, 0, 0, 465, 466, 467, 0, 468, - 469, 470, 471, 0, 472, 473, 474, 475, 476, 0, - 1483, 0, 479, 2858, 481, 482, 483, 484, 485, 486, - 487, 0, 0, 488, 0, 0, 489, 490, 491, 492, - 493, 494, 517, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 506, 507, 508, 509, 0, 0, 0, - 0, 111, 112, 113, 114, 115, 116, 117, 118, 0, - 119, 120, 121, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 123, 124, 0, 125, 126, 127, 0, 129, - 130, 131, 132, 133, 0, 135, 136, 0, 137, 138, - 139, 140, 141, 142, 0, 0, 143, 144, 145, 146, - 147, 148, 149, 0, 150, 151, 152, 153, 154, 0, - 0, 0, 156, 157, 158, 159, 160, 161, 0, 163, - 164, 165, 0, 166, 167, 168, 169, 170, 171, 0, - 0, 173, 174, 175, 176, 177, 178, 179, 180, 181, - 182, 183, 184, 185, 186, 187, 0, 188, 0, 189, - 190, 191, 192, 193, 194, 0, 195, 196, 197, 198, - 199, 0, 0, 200, 201, 202, 203, 204, 0, 205, - 206, 207, 0, 208, 209, 210, 0, 211, 212, 213, - 214, 215, 216, 217, 218, 219, 220, 221, 0, 222, - 0, 223, 224, 225, 226, 0, 227, 0, 228, 0, - 0, 0, 231, 232, 518, 0, 235, 0, 236, 0, - 237, 238, 0, 240, 0, 241, 242, 243, 244, 245, - 246, 247, 0, 249, 250, 251, 252, 0, 253, 254, - 255, 256, 257, 258, 259, 0, 260, 0, 262, 263, - 264, 265, 266, 267, 268, 269, 0, 270, 0, 271, - 0, 0, 274, 0, 276, 277, 278, 279, 280, 281, - 0, 0, 282, 0, 284, 0, 0, 286, 287, 288, - 289, 290, 291, 292, 293, 519, 295, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, 0, 317, 318, - 319, 320, 321, 0, 322, 323, 0, 325, 0, 326, - 327, 328, 329, 330, 331, 0, 332, 333, 0, 0, - 334, 335, 336, 0, 0, 337, 338, 339, 0, 341, - 0, 343, 344, 345, 346, 347, 348, 349, 0, 351, - 352, 353, 0, 0, 0, 0, 354, 355, 356, 0, - 358, 359, 360, 361, 362, 363, 0, 364, 365, 366, - 367, 368, 369, 0, 370, 0, 372, 373, 374, 375, - 376, 377, 0, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 0, 391, 392, 0, - 394, 395, 396, 397, 0, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 0, 0, 409, 410, 411, - 412, 413, 414, 415, 416, 417, 418, 0, 420, 421, - 422, 423, 0, 424, 425, 426, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 520, 437, 438, 439, 0, - 440, 441, 0, 442, 0, 444, 445, 446, 447, 448, - 0, 449, 450, 451, 0, 0, 452, 453, 454, 455, - 456, 0, 457, 458, 459, 460, 461, 462, 463, 464, - 0, 0, 465, 466, 467, 0, 468, 469, 470, 471, - 0, 472, 473, 474, 475, 476, 477, 478, 0, 479, - 0, 481, 482, 483, 484, 485, 486, 487, 0, 0, - 488, 0, 0, 489, 490, 491, 492, 493, 494, 495, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 837, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 838, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 839, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 840, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 841, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, - 506, 507, 508, 509, 1721, 0, 0, 0, 0, 0, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 950, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 111, 112, 113, 114, 115, 116, 117, - 118, 0, 119, 120, 121, 0, 0, 0, 1432, 0, - 0, 0, 0, 1433, 123, 124, 0, 125, 126, 127, - 1434, 129, 130, 131, 0, 1435, 1436, 1437, 1438, 0, - 137, 138, 139, 140, 141, 142, 0, 0, 143, 144, - 145, 146, 1439, 1440, 149, 0, 150, 151, 152, 153, - 0, 0, 1441, 0, 1442, 157, 158, 159, 160, 161, - 1443, 163, 164, 165, 0, 166, 167, 168, 169, 170, - 171, 0, 1444, 173, 174, 175, 176, 177, 178, 179, - 180, 181, 182, 1445, 184, 185, 1446, 187, 0, 188, - 0, 189, 190, 191, 192, 193, 194, 0, 195, 196, - 197, 198, 199, 0, 0, 200, 201, 950, 203, 204, - 0, 205, 206, 207, 0, 208, 209, 210, 0, 211, - 212, 213, 214, 0, 216, 217, 218, 219, 220, 0, - 0, 222, 0, 223, 224, 1447, 226, 0, 227, 0, - 228, 1448, 0, 1449, 231, 232, 0, 1450, 235, 0, - 236, 0, 0, 0, 239, 240, 0, 241, 242, 243, - 244, 245, 246, 247, 1451, 249, 250, 251, 252, 0, - 253, 254, 255, 256, 257, 258, 259, 0, 260, 1452, - 0, 263, 264, 265, 266, 267, 1453, 1454, 0, 1455, - 0, 271, 1456, 1457, 274, 1458, 276, 277, 278, 279, - 280, 281, 0, 0, 282, 1459, 284, 1460, 0, 286, - 287, 288, 289, 290, 291, 292, 293, 0, 295, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, 313, 314, 1462, 1463, - 1464, 318, 319, 320, 0, 0, 322, 323, 1465, 325, - 0, 0, 327, 1466, 329, 330, 331, 0, 332, 333, - 0, 0, 334, 335, 336, 0, 0, 337, 338, 0, - 1467, 341, 1468, 0, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 0, 0, 0, 0, 354, 355, - 0, 1469, 358, 359, 0, 361, 362, 363, 0, 364, - 365, 366, 367, 368, 369, 0, 370, 371, 372, 1470, - 374, 375, 376, 377, 0, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 0, 391, - 392, 1471, 394, 395, 396, 0, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 0, 1473, 409, - 410, 411, 412, 413, 414, 0, 416, 417, 418, 1475, - 420, 421, 1476, 423, 0, 424, 425, 426, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 0, 437, 0, - 439, 0, 440, 441, 0, 442, 1478, 444, 445, 446, - 447, 448, 0, 449, 1479, 1480, 0, 0, 452, 453, - 0, 455, 0, 0, 457, 458, 1481, 460, 461, 462, - 463, 464, 0, 0, 465, 466, 467, 0, 468, 469, - 470, 471, 0, 472, 473, 474, 475, 476, 0, 1483, - 0, 479, 1484, 481, 482, 483, 484, 485, 486, 487, - 0, 1, 488, 0, 0, 489, 490, 491, 492, 493, - 494, 2, 0, 3, 4, 0, 0, 0, 0, 0, - 0, 0, 506, 507, 508, 509, 0, 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, 0, 0, 8, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 9, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 10, 0, 11, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 839, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 841, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 1307, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 12, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 13, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 15, 0, 0, 0, - 0, 0, 16, 0, 0, 0, 0, 0, 0, 0, - 17, 0, 0, 0, 0, 0, 0, 0, 0, 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, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 1318, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 1667, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 0, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 1843, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 244, 245, 0, 246, 247, 248, 249, 250, 2210, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 129, 130, 0, 131, 132, + 133, 0, 135, 136, 137, 138, 139, 0, 141, 142, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 153, 154, 155, 0, 156, 157, 158, + 159, 160, 0, 0, 0, 162, 163, 164, 165, 166, + 167, 0, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 0, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 189, 190, 191, 192, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 207, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 220, 221, 222, 223, 224, 225, + 226, 0, 227, 0, 228, 229, 230, 231, 0, 232, + 0, 233, 0, 0, 0, 236, 237, 524, 0, 240, + 0, 241, 0, 242, 243, 244, 245, 0, 246, 247, + 248, 249, 250, 2225, 252, 0, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 0, 267, 268, 269, 270, 271, 272, 273, 274, 0, + 275, 0, 276, 0, 0, 279, 0, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 0, 289, 0, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 525, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 320, + 0, 322, 323, 324, 325, 326, 0, 327, 328, 0, + 330, 0, 331, 332, 333, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 344, 0, 346, 0, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 362, 0, 364, 365, 366, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 379, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 0, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 0, 417, 418, 419, 420, 421, 422, 423, + 424, 425, 0, 0, 427, 428, 429, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 526, 444, 445, 0, 0, 446, 447, 0, 448, + 0, 450, 451, 452, 453, 454, 0, 455, 456, 457, + 0, 0, 458, 459, 460, 461, 462, 0, 463, 464, + 465, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 483, 484, 0, 485, 0, 487, 488, 489, + 490, 491, 492, 493, 0, 0, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 501, 502, 503, 504, 505, + 506, 507, 508, 509, 510, 511, 512, 513, 514, 515, + 1472, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 1473, 0, 0, 0, 0, 1474, + 129, 130, 0, 131, 132, 133, 1475, 135, 136, 137, + 0, 1476, 1477, 1478, 1479, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 1480, 1481, + 155, 0, 156, 157, 158, 159, 0, 0, 1482, 0, + 1483, 163, 164, 165, 166, 167, 1484, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 1485, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 1486, + 190, 191, 1487, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 1059, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 0, + 221, 222, 223, 224, 225, 0, 0, 227, 0, 228, + 229, 1488, 231, 0, 232, 0, 233, 1489, 0, 1490, + 236, 237, 0, 1491, 240, 0, 241, 0, 0, 0, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 1492, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 1493, 0, 268, 269, 270, + 271, 272, 1494, 1495, 0, 1496, 0, 276, 1497, 1498, + 279, 1499, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 1500, 289, 1501, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 1502, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 1503, 1504, 1505, 323, 324, 325, + 0, 0, 327, 328, 1506, 330, 0, 0, 332, 1507, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 0, 1508, 346, 1509, 0, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 0, 1510, 364, + 365, 0, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 1511, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 1512, 400, + 401, 402, 1513, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 1514, 417, 418, + 419, 420, 421, 422, 1515, 424, 425, 0, 1516, 427, + 428, 1517, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 1518, 444, 0, 0, + 0, 446, 447, 0, 448, 1519, 450, 451, 452, 453, + 454, 0, 455, 1520, 1521, 0, 0, 458, 459, 0, + 461, 0, 0, 463, 464, 1522, 466, 467, 468, 469, + 470, 1523, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 0, 1524, 0, + 485, 1525, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 1472, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 512, 513, 514, 515, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 1473, 0, 0, 0, 0, 1474, + 129, 130, 0, 131, 132, 133, 1475, 135, 136, 137, + 0, 1476, 1477, 1478, 1479, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 1480, 1481, + 155, 0, 156, 157, 158, 159, 0, 0, 1482, 0, + 1483, 163, 164, 165, 166, 167, 1484, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 1485, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 1486, + 190, 191, 1487, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 1059, 208, 209, 0, 210, 211, 212, + 0, 1825, 214, 215, 0, 216, 217, 218, 219, 0, + 221, 222, 223, 224, 225, 0, 0, 227, 0, 228, + 229, 1488, 231, 0, 232, 0, 233, 1489, 0, 1490, + 236, 237, 0, 1491, 240, 0, 241, 0, 0, 0, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 1492, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 1493, 0, 268, 269, 270, + 271, 272, 1494, 1495, 0, 1496, 0, 276, 1497, 1498, + 279, 1499, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 1500, 289, 1501, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 1502, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 1503, 1504, 1505, 323, 324, 325, + 0, 0, 327, 328, 1506, 330, 0, 0, 332, 1507, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 0, 1508, 346, 1509, 0, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 0, 1510, 364, + 365, 0, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 1511, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 1512, 400, + 401, 402, 1513, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 1514, 417, 418, + 419, 420, 421, 422, 1515, 424, 425, 0, 1516, 427, + 428, 1517, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 1518, 444, 0, 0, + 0, 446, 447, 0, 448, 1519, 450, 451, 452, 453, + 454, 0, 455, 1520, 1521, 0, 0, 458, 459, 0, + 461, 0, 0, 463, 464, 1522, 466, 467, 468, 469, + 470, 1523, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 0, 1524, 0, + 485, 1525, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 3135, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 512, 513, 514, 515, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 2886, 0, 0, 0, 0, 2887, + 129, 130, 0, 131, 132, 133, 2888, 135, 136, 137, + 0, 1476, 2889, 1478, 1479, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 1480, 1481, + 155, 0, 156, 157, 158, 159, 0, 0, 2890, 0, + 2891, 163, 164, 165, 166, 167, 2892, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 2893, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 1486, + 190, 191, 1487, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 1059, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 0, + 221, 222, 223, 224, 225, 0, 0, 227, 0, 228, + 229, 1488, 231, 0, 232, 0, 233, 2894, 0, 2895, + 236, 237, 2896, 2897, 240, 0, 241, 0, 0, 0, + 244, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 2898, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 2899, 0, 268, 269, 270, + 271, 272, 1494, 1495, 0, 1496, 0, 276, 2900, 2901, + 279, 2902, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 2903, 289, 2904, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 3136, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 1503, 2906, 1505, 323, 324, 325, + 0, 0, 327, 328, 2908, 330, 0, 0, 332, 1507, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 0, 2910, 346, 2911, 0, + 349, 350, 351, 352, 353, 354, 355, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 0, 2912, 364, + 365, 0, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 377, 378, 1511, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 2913, 400, + 401, 402, 0, 404, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 2914, 417, 418, + 419, 420, 421, 422, 0, 424, 425, 0, 2916, 427, + 428, 1517, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 3137, 444, 0, 0, + 0, 446, 447, 0, 448, 2918, 450, 451, 452, 453, + 454, 0, 455, 1520, 1521, 0, 0, 458, 459, 0, + 461, 0, 0, 463, 464, 2919, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 0, 1524, 0, + 485, 2921, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 523, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 512, 513, 514, 515, 0, 0, 0, 0, 117, + 118, 119, 120, 121, 122, 123, 124, 0, 125, 126, + 127, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 129, 130, 0, 131, 132, 133, 0, 135, 136, 137, + 138, 139, 0, 141, 142, 0, 143, 144, 145, 146, + 147, 148, 0, 0, 149, 150, 151, 152, 153, 154, + 155, 0, 156, 157, 158, 159, 160, 0, 0, 0, + 162, 163, 164, 165, 166, 167, 0, 169, 170, 171, + 0, 172, 173, 174, 175, 176, 177, 0, 0, 179, + 180, 181, 182, 183, 184, 185, 186, 187, 188, 189, + 190, 191, 192, 193, 0, 194, 0, 195, 196, 197, + 198, 199, 200, 0, 0, 201, 202, 203, 204, 0, + 0, 205, 206, 207, 208, 209, 0, 210, 211, 212, + 0, 213, 214, 215, 0, 216, 217, 218, 219, 220, + 221, 222, 223, 224, 225, 226, 0, 227, 0, 228, + 229, 230, 231, 0, 232, 0, 233, 0, 0, 0, + 236, 237, 524, 0, 240, 0, 241, 0, 242, 243, + 0, 245, 0, 246, 247, 248, 249, 250, 251, 252, + 0, 254, 255, 256, 257, 0, 258, 259, 260, 261, + 262, 263, 264, 0, 265, 0, 267, 268, 269, 270, + 271, 272, 273, 274, 0, 275, 0, 276, 0, 0, + 279, 0, 281, 282, 283, 284, 285, 286, 0, 0, + 287, 0, 289, 0, 0, 291, 292, 293, 294, 295, + 296, 297, 298, 525, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, 317, 318, 319, 320, 0, 322, 323, 324, 325, + 326, 0, 327, 328, 0, 330, 0, 331, 332, 333, + 334, 335, 336, 0, 337, 338, 0, 0, 339, 340, + 341, 0, 0, 342, 343, 344, 0, 346, 0, 348, + 349, 350, 351, 352, 353, 354, 0, 356, 357, 358, + 359, 0, 0, 0, 0, 360, 361, 362, 0, 364, + 365, 366, 367, 368, 369, 0, 370, 371, 372, 373, + 374, 375, 0, 376, 0, 378, 379, 380, 381, 382, + 383, 0, 384, 385, 386, 387, 388, 389, 390, 391, + 392, 393, 394, 395, 396, 0, 397, 398, 0, 400, + 401, 402, 403, 0, 405, 406, 407, 408, 409, 410, + 411, 412, 413, 414, 415, 416, 0, 0, 417, 418, + 419, 420, 421, 422, 423, 424, 425, 0, 0, 427, + 428, 429, 430, 0, 431, 432, 433, 434, 435, 436, + 437, 438, 439, 440, 441, 442, 526, 444, 445, 0, + 0, 446, 447, 0, 448, 0, 450, 451, 452, 453, + 454, 0, 455, 456, 457, 0, 0, 458, 459, 460, + 461, 462, 0, 463, 464, 465, 466, 467, 468, 469, + 470, 0, 0, 471, 472, 473, 0, 474, 475, 476, + 477, 0, 478, 479, 480, 481, 482, 483, 484, 0, + 485, 0, 487, 488, 489, 490, 491, 492, 493, 0, + 0, 494, 0, 0, 495, 496, 497, 498, 499, 500, + 501, 502, 503, 504, 505, 506, 507, 508, 509, 510, + 511, 512, 513, 514, 515, 1762, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 117, 118, 119, 120, 121, 122, + 123, 124, 0, 125, 126, 127, 0, 0, 0, 1473, + 0, 0, 0, 0, 1474, 129, 130, 0, 131, 132, + 133, 1475, 135, 136, 137, 0, 1476, 1477, 1478, 1479, + 0, 143, 144, 145, 146, 147, 148, 0, 0, 149, + 150, 151, 152, 1480, 1481, 155, 0, 156, 157, 158, + 159, 0, 0, 1482, 0, 1483, 163, 164, 165, 166, + 167, 1484, 169, 170, 171, 0, 172, 173, 174, 175, + 176, 177, 0, 1485, 179, 180, 181, 182, 183, 184, + 185, 186, 187, 188, 1486, 190, 191, 1487, 193, 0, + 194, 0, 195, 196, 197, 198, 199, 200, 0, 0, + 201, 202, 203, 204, 0, 0, 205, 206, 1059, 208, + 209, 0, 210, 211, 212, 0, 213, 214, 215, 0, + 216, 217, 218, 219, 0, 221, 222, 223, 224, 225, + 0, 0, 227, 0, 228, 229, 1488, 231, 0, 232, + 0, 233, 1489, 0, 1490, 236, 237, 0, 1491, 240, + 0, 241, 0, 0, 0, 244, 245, 0, 246, 247, + 248, 249, 250, 251, 252, 1492, 254, 255, 256, 257, + 0, 258, 259, 260, 261, 262, 263, 264, 0, 265, + 1493, 0, 268, 269, 270, 271, 272, 1494, 1495, 0, + 1496, 0, 276, 1497, 1498, 279, 1499, 281, 282, 283, + 284, 285, 286, 0, 0, 287, 1500, 289, 1501, 0, + 291, 292, 293, 294, 295, 296, 297, 298, 0, 300, + 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, 314, 315, 316, 317, 318, 319, 1503, + 1504, 1505, 323, 324, 325, 0, 0, 327, 328, 1506, + 330, 0, 0, 332, 1507, 334, 335, 336, 0, 337, + 338, 0, 0, 339, 340, 341, 0, 0, 342, 343, + 0, 1508, 346, 1509, 0, 349, 350, 351, 352, 353, + 354, 355, 356, 357, 358, 359, 0, 0, 0, 0, + 360, 361, 0, 1510, 364, 365, 0, 367, 368, 369, + 0, 370, 371, 372, 373, 374, 375, 0, 376, 377, + 378, 1511, 380, 381, 382, 383, 0, 384, 385, 386, + 387, 388, 389, 390, 391, 392, 393, 394, 395, 396, + 0, 397, 398, 1512, 400, 401, 402, 0, 404, 405, + 406, 407, 408, 409, 410, 411, 412, 413, 414, 415, + 416, 0, 1514, 417, 418, 419, 420, 421, 422, 0, + 424, 425, 0, 1516, 427, 428, 1517, 430, 0, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 0, 444, 0, 0, 0, 446, 447, 0, 448, + 1519, 450, 451, 452, 453, 454, 0, 455, 1520, 1521, + 0, 0, 458, 459, 0, 461, 0, 0, 463, 464, + 1522, 466, 467, 468, 469, 470, 0, 0, 471, 472, + 473, 0, 474, 475, 476, 477, 0, 478, 479, 480, + 481, 482, 0, 1524, 0, 485, 1525, 487, 488, 489, + 490, 491, 492, 493, 0, 1, 494, 0, 0, 495, + 496, 497, 498, 499, 500, 2, 0, 3, 4, 0, + 0, 0, 0, 1, 0, 0, 512, 513, 514, 515, + 0, 0, 0, 2, 0, 6, 0, 0, 0, 0, + 0, 0, 0, 0, 7, 0, 0, 0, 0, 0, + 0, 0, 0, 6, 0, 0, 0, 0, 8, 0, + 0, 0, 7, 0, 0, 0, 0, 0, 0, 10, + 0, 0, 0, 0, 0, 0, 8, 0, 0, 0, + 0, 11, 0, 746, 0, 0, 0, 10, 0, 0, + 0, 0, 0, 0, 13, 0, 0, 0, 0, 11, + 0, 746, 0, 0, 0, 0, 0, 0, 0, 14, + 15, 0, 13, 0, 0, 0, 0, 0, 0, 0, + 747, 0, 0, 0, 0, 0, 18, 14, 15, 0, + 0, 0, 0, 0, 19, 0, 0, 0, 747, 0, + 0, 0, 0, 0, 18, 0, 0, 0, 0, 0, + 22, 0, 19, 0, 23, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 0, 22, 0, 0, 0, 23, 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, 24, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 25, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 25, 0, 0, 0, 0, 0, + 0, 0, 26, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, + 26, 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, 25, 26, 27, 0, 0, 0, 0, 0, - 28, 0, 0, 29, 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, 30, 0, 0, 0, 0, 0, 0, 0, - 0, 31, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 27, 28, 29, + 0, 0, 0, 0, 0, 30, 0, 0, 31, 0, + 0, 0, 0, 0, 0, 27, 28, 29, 0, 0, + 0, 0, 0, 30, 0, 0, 31, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, + 0, 0, 0, 0, 0, 0, 33, 0, 0, 0, 0, 0, 0, 0, 0, 32, 0, 0, 0, 0, - 0, 0, 33, 0, 0, 0, 34, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 35, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 36, 0, - 0, 0, 37, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 38, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 39, 0, 0, 0, 0, - 0, 41, 0, 0, 0, 0, 42, 0, 0, 0, - 575, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 43, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 34, 0, 33, 0, 0, 0, 0, 35, + 0, 0, 0, 36, 0, 0, 0, 0, 0, 0, + 34, 0, 0, 37, 0, 0, 0, 35, 0, 0, + 0, 36, 0, 0, 0, 38, 0, 0, 0, 39, + 0, 37, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 38, 0, 0, 0, 39, 0, 40, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 44, 0, 0, 0, - 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, - 576 + 0, 0, 41, 0, 0, 0, 0, 40, 43, 0, + 0, 0, 0, 44, 0, 0, 0, 748, 0, 0, + 41, 0, 0, 0, 0, 0, 43, 0, 0, 45, + 0, 44, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 45, 0, 0, + 0, 0, 0, 46, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 0, 0, 749, 0, 0, + 0, 46, 0, 0, 0, 0, 0, 0, 0, 0, + 0, 0, 0, 0, 0, 47 }; static const yytype_int16 yycheck[] = { - 7, 730, 510, 0, 570, 0, 0, 0, 662, 721, - 0, 735, 707, 636, 21, 45, 0, 576, 1259, 0, - 881, 1373, 0, 69, 0, 1647, 1098, 861, 1112, 740, - 0, 73, 7, 1085, 18, 872, 1090, 768, 1124, 685, - 1110, 1610, 2172, 36, 18, 2117, 21, 1087, 923, 73, - 872, 1421, 14, 1933, 7, 1046, 872, 970, 1529, 772, - 2086, 1241, 2088, 1079, 71, 72, 861, 2581, 21, 2107, - 1279, 1934, 2220, 2221, 2222, 1115, 872, 980, 1013, 575, - 1752, 1563, 1564, 1018, 2579, 576, 37, 632, 571, 721, - 2544, 723, 637, 725, 2544, 0, 71, 72, 2052, 2168, - 21, 94, 0, 44, 2106, 0, 0, 2594, 2418, 0, - 33, 24, 0, 2234, 1771, 0, 0, 978, 71, 72, - 0, 629, 0, 0, 0, 0, 2500, 0, 0, 0, - 2504, 0, 0, 1768, 13, 14, 0, 2601, 0, 9, - 0, 0, 23, 24, 0, 1665, 1614, 21, 13, 14, - 71, 72, 63, 104, 5, 1667, 5, 1907, 1908, 948, - 1874, 890, 13, 14, 63, 55, 5, 5, 1918, 677, - 5, 11, 1922, 5, 5, 964, 16, 9, 13, 14, - 117, 599, 2251, 5, 5, 2908, 5, 2245, 944, 945, - 1024, 702, 13, 14, 13, 14, 702, 71, 72, 5, - 172, 5, 26, 697, 122, 961, 46, 13, 14, 5, - 119, 26, 45, 171, 5, 100, 100, 32, 9, 45, - 5, 5, 2241, 2895, 870, 5, 190, 108, 2297, 2298, - 74, 2300, 5, 63, 171, 2369, 4, 30, 2751, 2911, - 80, 9, 5, 5, 5, 38, 5, 173, 180, 5, - 873, 5, 139, 107, 30, 30, 287, 202, 74, 171, - 107, 104, 38, 38, 147, 1099, 11, 289, 1102, 1103, - 9, 16, 290, 3, 4, 5, 295, 71, 72, 9, - 122, 171, 11, 295, 107, 63, 15, 16, 2797, 4, - 226, 40, 312, 122, 9, 2867, 192, 312, 3201, 226, - 1029, 117, 861, 2462, 124, 2464, 244, 117, 137, 275, - 119, 172, 3, 137, 132, 2463, 82, 46, 40, 245, - 385, 3073, 137, 376, 168, 2351, 1567, 93, 165, 874, - 117, 977, 278, 1208, 217, 3232, 272, 180, 82, 40, - 192, 886, 1217, 34, 35, 272, 163, 90, 364, 93, - 365, 80, 101, 11, 103, 3308, 105, 15, 16, 11, - 163, 80, 75, 30, 64, 451, 303, 148, 864, 30, - 212, 108, 861, 3399, 74, 412, 313, 314, 315, 106, - 190, 1507, 2732, 2513, 2734, 33, 129, 117, 46, 2903, - 1516, 3294, 13, 14, 3297, 355, 108, 1388, 2852, 427, - 2914, 337, 1474, 2075, 289, 289, 3265, 1844, 500, 1312, - 2905, 59, 295, 450, 3440, 341, 2488, 413, 199, 511, - 1074, 1075, 80, 383, 369, 511, 3098, 230, 80, 272, - 1421, 11, 1078, 378, 108, 15, 16, 119, 1875, 376, - 108, 2510, 3395, 2512, 390, 356, 511, 308, 476, 272, - 386, 2514, 378, 175, 271, 351, 2410, 356, 274, 386, - 1701, 3358, 515, 279, 274, 132, 284, 425, 413, 293, - 279, 132, 357, 357, 175, 983, 413, 3200, 293, 170, - 511, 172, 448, 327, 2643, 322, 506, 413, 218, 511, - 3393, 506, 279, 511, 272, 511, 515, 326, 3357, 3071, - 213, 11, 354, 515, 3017, 15, 16, 445, 997, 446, - 240, 327, 448, 425, 3003, 369, 3005, 191, 463, 451, - 457, 448, 3031, 250, 206, 2659, 356, 343, 389, 353, - 448, 355, 265, 260, 527, 1024, 473, 463, 353, 511, - 240, 3050, 500, 1267, 453, 166, 576, 511, 2611, 2607, - 1245, 2620, 1900, 1901, 1902, 513, 1926, 1051, 441, 383, - 497, 2441, 400, 401, 307, 1076, 413, 1480, 383, 576, - 1076, 2590, 3295, 511, 511, 120, 1756, 410, 356, 422, - 3332, 2052, 469, 2933, 410, 509, 319, 511, 473, 473, - 272, 575, 274, 2307, 1803, 1836, 2253, 627, 442, 3086, - 393, 5, 576, 2123, 515, 383, 448, 1848, 2090, 616, - 1099, 2246, 2124, 1102, 1103, 2365, 515, 393, 393, 3073, - 627, 511, 1040, 3073, 3296, 249, 442, 512, 512, 514, - 514, 451, 511, 675, 3123, 451, 475, 1878, 2768, 454, - 510, 616, 2110, 1695, 1885, 2116, 511, 895, 3022, 194, - 274, 675, 627, 3027, 2775, 662, 663, 1446, 497, 425, - 511, 707, 511, 616, 460, 913, 2976, 507, 519, 917, - 519, 475, 511, 1576, 627, 418, 511, 1466, 685, 511, - 511, 425, 1923, 1439, 1440, 3199, 1927, 509, 2802, 511, - 511, 513, 511, 497, 0, 616, 2810, 694, 611, 694, - 694, 694, 1786, 511, 694, 511, 627, 511, 1464, 683, - 694, 1500, 1278, 694, 1955, 511, 694, 1283, 694, 3206, - 511, 728, 729, 1289, 694, 732, 511, 511, 609, 680, - 611, 511, 1818, 1379, 502, 503, 1295, 1612, 511, 1393, - 2888, 664, 665, 3207, 667, 686, 697, 2749, 511, 511, - 511, 1931, 511, 627, 2634, 511, 1410, 511, 473, 514, - 641, 161, 507, 502, 503, 772, 132, 171, 1524, 1525, - 461, 473, 502, 503, 515, 504, 505, 506, 507, 528, - 41, 244, 497, 532, 461, 666, 535, 502, 503, 694, - 509, 403, 3306, 274, 513, 497, 694, 3292, 1294, 694, - 694, 2755, 1285, 694, 1295, 2668, 694, 1710, 2671, 1712, - 2673, 1300, 1301, 451, 694, 5, 694, 694, 694, 694, - 244, 694, 694, 694, 2893, 694, 694, 1556, 1557, 1558, - 694, 2711, 694, 627, 694, 694, 161, 237, 694, 511, - 82, 1386, 3258, 3259, 502, 503, 504, 505, 506, 507, - 26, 93, 108, 465, 861, 3350, 32, 120, 26, 1367, - 1506, 1507, 869, 870, 32, 126, 30, 2076, 875, 618, - 1516, 878, 879, 511, 881, 882, 883, 884, 3332, 351, - 864, 3160, 3332, 3162, 1530, 244, 286, 2559, 274, 2561, - 897, 1725, 149, 279, 3310, 509, 1887, 904, 475, 513, - 506, 1735, 183, 3033, 1738, 3419, 369, 513, 274, 313, - 314, 315, 237, 1559, 173, 922, 923, 924, 284, 1794, - 497, 85, 897, 191, 504, 505, 506, 507, 4, 904, - 94, 194, 900, 9, 511, 1926, 943, 202, 906, 2410, - 500, 475, 475, 200, 897, 369, 2428, 132, 978, 335, - 37, 904, 3, 513, 118, 962, 53, 3236, 244, 40, - 1895, 137, 711, 497, 497, 244, 973, 974, 975, 137, - 977, 978, 376, 980, 159, 3013, 897, 2329, 511, 60, - 861, 26, 445, 904, 509, 1819, 245, 32, 513, 258, - 259, 872, 502, 503, 504, 505, 506, 507, 747, 748, - 749, 750, 1009, 251, 2352, 2353, 2354, 2355, 1720, 413, - 369, 430, 2003, 1579, 2255, 1057, 1058, 177, 1060, 1026, - 1027, 445, 37, 897, 1819, 106, 190, 82, 244, 413, - 904, 509, 1521, 1057, 1058, 513, 1060, 463, 93, 203, - 921, 372, 446, 2223, 1533, 3175, 1535, 2927, 511, 71, - 72, 2723, 1059, 457, 510, 336, 1063, 1064, 1753, 1754, - 1755, 1693, 511, 11, 1775, 521, 1073, 1074, 1075, 473, - 1801, 1078, 1561, 2544, 1805, 511, 2092, 1808, 2094, 176, - 240, 511, 341, 369, 365, 172, 445, 511, 1720, 274, - 369, 1098, 137, 497, 279, 166, 193, 1810, 46, 284, - 1051, 198, 3250, 897, 373, 374, 987, 511, 1740, 4, - 904, 341, 2196, 1745, 9, 506, 997, 293, 510, 378, - 201, 208, 513, 4, 511, 293, 161, 876, 9, 521, - 4, 166, 80, 8, 1015, 9, 11, 224, 1826, 236, - 15, 16, 1830, 1024, 2385, 1833, 511, 234, 378, 650, - 651, 652, 511, 369, 413, 2146, 511, 172, 905, 445, - 1122, 511, 909, 1114, 2204, 1731, 445, 161, 171, 250, - 1736, 46, 166, 349, 511, 1664, 1665, 353, 53, 260, - 509, 349, 511, 413, 513, 353, 1378, 8, 1380, 1381, - 11, 272, 147, 208, 15, 16, 13, 14, 19, 20, - 21, 1208, 237, 511, 463, 80, 161, 383, 2262, 224, - 1217, 166, 504, 512, 2268, 383, 515, 341, 417, 234, - 419, 2082, 26, 304, 512, 511, 511, 515, 32, 445, - 13, 14, 511, 463, 1241, 399, 1725, 1883, 402, 664, - 665, 512, 667, 237, 515, 511, 1735, 432, 293, 1738, - 512, 286, 1259, 515, 378, 13, 14, 417, 443, 419, - 1819, 1258, 217, 1258, 1258, 171, 512, 2637, 1258, 515, - 145, 497, 1279, 360, 355, 2178, 2179, 2180, 454, 494, - 512, 3411, 237, 515, 2755, 445, 454, 511, 1295, 413, - 377, 388, 286, 512, 223, 511, 515, 378, 417, 512, - 419, 176, 515, 2544, 349, 1312, 512, 289, 353, 515, - 1294, 1318, 512, 494, 512, 515, 397, 515, 193, 1965, - 513, 1295, 6, 198, 3396, 9, 3398, 147, 13, 14, - 1819, 286, 16, 137, 512, 1084, 3362, 515, 383, 463, - 295, 161, 2178, 513, 2180, 360, 166, 31, 1097, 515, - 512, 35, 3378, 515, 512, 170, 59, 515, 1365, 1366, - 512, 236, 377, 515, 1371, 512, 1373, 3439, 515, 1118, - 511, 1378, 1379, 1380, 1381, 512, 2367, 198, 515, 512, - 2371, 512, 515, 473, 515, 475, 1393, 1394, 2291, 13, - 14, 1398, 2087, 1400, 2040, 512, 1403, 217, 1373, 512, - 221, 1408, 515, 1410, 1411, 512, 1413, 3433, 515, 454, - 1417, 512, 3438, 417, 515, 419, 291, 237, 512, 512, - 1373, 515, 515, 1398, 512, 1400, 512, 515, 1403, 515, - 1311, 512, 512, 1408, 1315, 515, 1411, 512, 1413, 417, - 515, 419, 1417, 2527, 2175, 1398, 2177, 1400, 13, 14, - 1403, 1940, 1373, 512, 512, 1408, 2526, 515, 1411, 2511, - 1413, 2510, 512, 2512, 1417, 515, 286, 1474, 171, 177, - 291, 2525, 2558, 2527, 699, 295, 701, 1398, 171, 1400, - 294, 3153, 1403, 3155, 13, 14, 441, 1408, 132, 293, - 1411, 512, 1413, 59, 515, 512, 1417, 2151, 515, 1506, - 1507, 511, 2415, 2416, 2574, 414, 1255, 494, 1515, 1516, - 1503, 1260, 223, 388, 512, 159, 1523, 515, 152, 1503, - 2244, 349, 1503, 1530, 1398, 1503, 1400, 1503, 512, 1403, - 13, 14, 240, 1503, 1408, 13, 14, 1411, 152, 1413, - 13, 14, 152, 1417, 152, 349, 13, 14, 1555, 353, - 13, 14, 1559, 171, 509, 1562, 2393, 2418, 513, 152, - 1567, 1568, 1569, 1570, 1571, 1572, 1573, 1574, 1575, 1576, - 40, 2393, 2394, 1580, 1581, 511, 2392, 2818, 1585, 383, - 13, 14, 1589, 364, 365, 1592, 1593, 1594, 1595, 1596, - 1597, 1598, 1599, 1600, 3163, 274, 1603, 2393, 2394, 2395, - 2089, 40, 3073, 1610, 1398, 1612, 1400, 463, 1650, 1403, - 313, 314, 315, 89, 1408, 13, 14, 1411, 152, 1413, - 495, 441, 477, 1417, 1631, 152, 1650, 502, 503, 504, - 505, 506, 507, 2122, 2123, 279, 13, 14, 2541, 152, - 284, 513, 2545, 152, 2547, 26, 1653, 1396, 13, 14, - 454, 32, 188, 189, 26, 1629, 289, 1664, 1665, 425, - 32, 369, 13, 14, 511, 177, 13, 14, 13, 14, - 13, 14, 171, 376, 13, 14, 511, 498, 499, 500, - 511, 502, 503, 504, 505, 506, 507, 13, 14, 509, - 202, 415, 512, 513, 1701, 313, 314, 315, 13, 14, - 13, 14, 879, 1710, 512, 1712, 13, 14, 511, 417, - 413, 419, 219, 2279, 364, 365, 224, 2769, 364, 365, - 262, 263, 258, 259, 364, 365, 2285, 299, 240, 511, - 2971, 224, 2816, 224, 373, 374, 511, 445, 3360, 1620, - 456, 457, 3364, 446, 40, 1752, 296, 177, 235, 1756, - 2479, 511, 1759, 1760, 457, 5, 137, 5, 376, 3373, - 3374, 511, 3114, 8, 323, 137, 11, 3405, 3406, 511, - 15, 16, 202, 511, 19, 20, 21, 1026, 1027, 511, - 2852, 5, 294, 5, 511, 12, 5, 1794, 432, 511, - 17, 5, 148, 9, 511, 413, 1803, 1790, 3420, 443, - 474, 511, 29, 1810, 1811, 2891, 301, 104, 511, 515, - 240, 515, 1819, 512, 40, 219, 43, 44, 166, 383, - 286, 235, 166, 284, 59, 511, 425, 93, 446, 1836, - 511, 425, 3073, 1840, 1841, 515, 1843, 373, 374, 457, - 425, 1848, 1849, 1850, 1851, 1852, 1853, 1854, 2502, 2724, - 1857, 1858, 1859, 1860, 1861, 1862, 1863, 1864, 1865, 1866, - 59, 3332, 59, 265, 294, 1872, 1873, 1748, 425, 1876, - 2565, 1878, 108, 2584, 521, 102, 1883, 425, 1885, 1628, - 2604, 1630, 2613, 37, 425, 473, 274, 152, 42, 100, - 274, 1640, 511, 511, 40, 274, 274, 3, 1905, 511, - 2389, 2390, 1909, 13, 1911, 417, 513, 419, 1915, 147, - 512, 512, 293, 3265, 171, 512, 1923, 512, 224, 515, - 1927, 293, 1929, 161, 1931, 512, 1933, 511, 166, 1678, - 442, 512, 512, 445, 1909, 470, 224, 511, 1819, 369, - 1915, 281, 511, 281, 1825, 515, 513, 101, 1955, 460, - 511, 511, 39, 198, 469, 3307, 1909, 511, 1965, 1966, - 511, 98, 1915, 9, 511, 423, 511, 513, 349, 423, - 11, 351, 353, 2807, 423, 515, 221, 349, 510, 217, - 279, 353, 521, 520, 515, 511, 511, 417, 1909, 419, - 425, 180, 162, 171, 1915, 515, 512, 217, 2005, 237, - 451, 515, 383, 2010, 2011, 3357, 436, 290, 226, 265, - 2040, 383, 442, 3083, 312, 445, 386, 924, 172, 312, - 180, 1770, 1771, 1772, 1773, 1774, 180, 515, 2035, 2036, - 515, 219, 512, 2040, 171, 1909, 511, 274, 332, 226, - 226, 1915, 295, 287, 463, 152, 291, 2054, 286, 511, - 2057, 511, 2059, 152, 208, 2544, 152, 295, 473, 152, - 40, 274, 40, 59, 289, 11, 289, 171, 2075, 2076, - 224, 40, 166, 454, 512, 2082, 512, 511, 2085, 511, - 234, 2942, 454, 512, 512, 180, 511, 2962, 2819, 2820, - 166, 3332, 3, 3, 39, 2102, 509, 509, 2664, 510, - 425, 425, 425, 425, 512, 512, 512, 512, 2115, 40, - 2085, 3197, 515, 513, 268, 1909, 512, 1024, 494, 520, - 494, 1915, 512, 512, 171, 2132, 2133, 2102, 494, 60, - 512, 155, 2085, 425, 249, 289, 470, 512, 511, 40, - 511, 59, 2149, 511, 2151, 500, 515, 3138, 511, 2102, - 1899, 511, 290, 2160, 496, 290, 515, 448, 243, 313, - 59, 59, 265, 425, 2085, 274, 320, 152, 202, 152, - 511, 2178, 2179, 2180, 425, 106, 313, 314, 315, 152, - 2669, 2102, 425, 425, 425, 512, 40, 351, 1365, 1366, - 281, 1098, 281, 511, 1371, 511, 515, 287, 512, 473, - 40, 289, 152, 441, 7, 8, 360, 279, 512, 12, - 2217, 2085, 171, 59, 17, 185, 2223, 512, 21, 2249, - 23, 24, 2788, 377, 511, 28, 29, 30, 2102, 511, - 33, 512, 512, 36, 37, 80, 166, 40, 509, 376, - 43, 44, 2239, 512, 2239, 2239, 143, 198, 2255, 2239, - 171, 3335, 512, 498, 499, 500, 511, 502, 503, 504, - 505, 506, 507, 512, 300, 512, 2273, 521, 71, 72, - 201, 357, 512, 3176, 290, 180, 413, 511, 2285, 152, - 512, 511, 515, 512, 2291, 175, 436, 515, 40, 40, - 511, 94, 446, 86, 451, 512, 433, 451, 101, 102, - 103, 104, 105, 512, 2185, 512, 512, 511, 171, 446, - 512, 511, 515, 511, 515, 542, 512, 472, 2807, 250, - 457, 512, 2329, 2812, 198, 510, 457, 3051, 512, 260, - 510, 512, 515, 512, 289, 512, 473, 512, 500, 59, - 512, 272, 74, 512, 512, 572, 512, 473, 80, 204, - 117, 2232, 1259, 40, 2329, 511, 226, 88, 191, 279, - 497, 93, 2369, 279, 2923, 513, 513, 425, 2249, 513, - 2859, 2860, 924, 304, 511, 500, 2329, 513, 2385, 513, - 513, 513, 425, 513, 513, 117, 513, 119, 2418, 513, - 513, 513, 513, 513, 2401, 513, 513, 513, 510, 513, - 2407, 2408, 513, 40, 2285, 513, 274, 512, 2329, 510, - 2456, 2418, 2293, 513, 2295, 513, 2369, 107, 2299, 511, - 2301, 2451, 2429, 473, 355, 2432, 40, 2434, 151, 513, - 513, 3334, 513, 513, 2441, 2442, 3165, 512, 2445, 2446, - 513, 289, 513, 2450, 2451, 3140, 513, 378, 171, 513, - 2457, 512, 511, 9, 350, 511, 511, 334, 515, 686, - 512, 515, 3323, 59, 512, 2472, 397, 191, 198, 456, - 510, 515, 2502, 91, 206, 2482, 343, 2470, 3124, 512, - 3126, 511, 3136, 2476, 152, 40, 2470, 513, 124, 2470, - 512, 2240, 2470, 152, 2470, 2502, 40, 512, 40, 8, - 2470, 365, 11, 2252, 2253, 2254, 15, 16, 40, 365, - 19, 20, 21, 512, 511, 40, 3139, 511, 2267, 3142, - 515, 2270, 451, 511, 309, 511, 2275, 248, 60, 279, - 190, 74, 451, 436, 2541, 511, 292, 2544, 2545, 74, - 2547, 80, 274, 3267, 9, 512, 1098, 279, 512, 511, - 367, 512, 2559, 510, 2561, 59, 510, 93, 8, 133, - 500, 11, 3257, 272, 3276, 15, 16, 1474, 40, 511, - 289, 436, 3061, 3062, 106, 107, 292, 292, 511, 204, - 512, 456, 289, 289, 3073, 117, 2583, 2580, 2583, 2583, - 313, 314, 315, 2583, 383, 327, 46, 3251, 512, 512, - 512, 122, 364, 53, 512, 448, 25, 148, 2357, 2358, - 2359, 343, 36, 2606, 297, 364, 2609, 694, 2528, 1718, - 2173, 2541, 2480, 3167, 2631, 3307, 2656, 2634, 3261, 3410, - 80, 1810, 2812, 3282, 3388, 3044, 3336, 3345, 3381, 1085, - 3100, 2179, 2166, 175, 2651, 2652, 2595, 3334, 2163, 2656, - 3343, 3331, 2659, 376, 2520, 2547, 1278, 2238, 1256, 2291, - 1567, 2923, 2576, 2544, 2149, 896, 924, 2115, 1067, 201, - 1689, 1066, 1849, 1850, 1851, 1852, 1853, 1854, 1045, 2686, - 1857, 1858, 1859, 1860, 1861, 1862, 1863, 1864, 1865, 1866, - 413, 896, 3316, 2700, 2350, 2576, 1653, 2132, 2705, 2706, - 3238, 3128, 1688, 2710, 2711, 1069, 2659, 1259, 2715, 21, - 442, 2718, 2719, 2102, 1921, 2745, 2723, 2724, 250, 451, - 2727, 3229, 2966, 446, 2731, 528, 176, 627, 260, 532, - 2659, 2738, 535, 874, 457, 616, 2329, 2328, 3304, 542, - 272, 2377, 274, 193, 3065, 1936, 1819, 2496, 198, 3242, - 473, 3241, 2012, 872, 2408, 2116, 2731, 2058, 1967, 872, - 2055, 2738, 1394, 2014, 2470, 2425, 3227, 570, 571, 572, - 872, 610, 304, 1295, 497, 2782, 1929, 1315, 2731, 1629, - 2661, 1630, 291, 2790, 2481, 2216, 236, 94, 511, -1, - 872, -1, -1, 872, 1701, -1, -1, -1, 872, -1, - -1, -1, 2809, 872, 872, -1, 609, 610, 611, -1, - 2731, 2818, -1, 616, -1, 618, -1, -1, 1045, -1, - -1, -1, -1, 355, 627, -1, -1, -1, 631, 632, - -1, -1, -1, 636, 637, -1, 2585, 37, -1, -1, - 1098, 291, 42, 3332, -1, 2852, 378, 650, 651, 652, - -1, -1, -1, -1, 2735, -1, -1, 2731, 2035, 2036, - -1, 664, 665, -1, 667, 397, -1, 399, -1, -1, - 402, -1, -1, -1, -1, -1, -1, 680, -1, -1, - -1, -1, -1, 686, -1, -1, -1, -1, -1, -1, - 2897, -1, -1, -1, 697, 2644, 2645, 2646, 2647, -1, - -1, 101, -1, -1, -1, -1, -1, -1, 711, -1, - -1, -1, 2942, 2906, -1, -1, 2923, -1, -1, 2926, - 2927, -1, 1474, -1, -1, -1, -1, -1, 151, 1836, - 733, -1, -1, -1, -1, 2942, -1, 2731, 388, -1, - -1, 1848, -1, -1, 747, 748, 749, 750, 171, -1, - -1, -1, -1, -1, -1, 2962, -1, -1, -1, -1, - -1, -1, -1, -1, 2971, -1, -1, -1, -1, 2976, - -1, 1878, 172, -1, -1, -1, -1, -1, 1885, 511, - 2987, 2988, -1, -1, 2991, -1, 2993, -1, -1, 498, - 499, 500, -1, 502, 503, 504, 505, 506, 507, -1, - -1, 1259, -1, -1, -1, -1, -1, -1, 208, -1, - -1, 3018, -1, -1, -1, 1567, 1923, -1, -1, -1, - 1927, 2902, -1, -1, 224, -1, 1933, -1, -1, -1, - -1, -1, -1, -1, 234, 3042, -1, -1, -1, -1, - -1, -1, 2923, -1, -1, 495, -1, 2796, 1955, -1, - -1, -1, 502, 503, 504, 505, 506, 507, -1, 1286, - -1, 1288, -1, -1, -1, -1, 3073, -1, 268, -1, - 873, 874, -1, 876, -1, -1, 879, -1, -1, -1, - -1, -1, 885, 886, -1, -1, -1, -1, 891, 289, - 313, 314, 315, -1, 897, 3088, 3089, -1, -1, -1, - -1, 904, -1, 3110, -1, -1, -1, 3114, -1, -1, - -1, 914, -1, 313, 0, -1, -1, 3124, 921, 3126, - 320, 3128, -1, 3004, -1, 3132, -1, 3134, 931, 3136, - -1, -1, 18, -1, -1, 21, -1, -1, -1, 3114, - -1, 3148, -1, -1, -1, -1, 3153, -1, 3155, 1701, - 36, -1, -1, 376, -1, 8, 3163, -1, 11, 45, - 360, 3114, 15, 16, -1, -1, 19, 20, 21, 3176, - -1, 3164, -1, 3166, 3181, -1, -1, 377, -1, 982, - -1, -1, -1, 36, -1, 71, 72, 73, -1, -1, - 413, -1, 3073, 3114, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 94, -1, - -1, -1, 3205, -1, -1, -1, 1474, -1, -1, -1, - -1, 3228, -1, 446, -1, 1028, -1, 1030, 3221, -1, - -1, 3238, 3225, -1, 457, -1, -1, -1, 924, -1, - -1, -1, 1045, 1046, 3251, -1, 446, -1, 1051, -1, - 473, 451, -1, -1, -1, -1, -1, -1, 3265, 8, - -1, -1, 11, -1, -1, -1, 15, 16, 2445, 2446, - 1073, -1, -1, -1, 497, -1, 3283, -1, -1, -1, - -1, 1084, 1085, -1, 1836, -1, 171, -1, 511, -1, - 3265, -1, -1, -1, 1097, -1, 1848, 46, -1, -1, - 3307, -1, -1, -1, 53, -1, -1, -1, -1, 1567, - -1, 1114, 3265, -1, -1, 1118, 3323, 3198, -1, -1, - 1123, -1, -1, -1, -1, 3332, 1878, 3334, -1, -1, - -1, 80, 3307, 1885, -1, -1, -1, -1, -1, -1, - -1, -1, 3091, -1, 3265, -1, -1, -1, 2255, -1, - 3357, 924, -1, -1, 3307, -1, 3349, -1, -1, -1, - -1, -1, 3111, -1, -1, 3372, 3373, 3374, 221, -1, - -1, 1923, -1, -1, -1, 1927, -1, -1, -1, -1, - -1, 1933, 3357, 3376, -1, -1, 3307, -1, -1, -1, - -1, -1, 3399, -1, -1, -1, 145, -1, -1, -1, - -1, -1, -1, 1955, 3357, -1, -1, -1, -1, -1, - -1, -1, 1098, -1, -1, -1, -1, -1, 0, -1, - -1, -1, -1, -1, -1, 3174, -1, 176, 313, 314, - 315, -1, -1, 3440, -1, -1, 3357, -1, 291, -1, - -1, -1, -1, 1701, 193, -1, -1, -1, -1, 198, - 1253, 3332, 1255, -1, 2631, 3204, -1, 1260, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 1271, 1696, - -1, -1, -1, -1, -1, 1278, -1, -1, 2385, 1706, - 1283, 1708, 1285, 1286, 1711, 1288, 1289, 236, 3369, -1, - 1717, 376, 1719, -1, -1, -1, -1, -1, -1, -1, - -1, 40, -1, -1, -1, 1732, -1, -1, -1, -1, - 1737, -1, 1315, -1, 1741, 1742, 1743, 1744, 100, 1746, - 1747, 60, -1, -1, -1, 1098, -1, -1, 413, -1, - -1, -1, 8, -1, 2441, 11, -1, -1, 171, 15, - 16, -1, 291, 19, 20, 21, -1, -1, -1, -1, + 7, 0, 0, 516, 7, 89, 90, 895, 0, 16, + 87, 0, 835, 741, 920, 883, 23, 0, 858, 809, + 23, 990, 0, 0, 1221, 1570, 904, 981, 47, 1416, + 23, 1220, 7, 0, 16, 1689, 0, 1195, 0, 0, + 1234, 1032, 20, 20, 925, 972, 749, 1208, 23, 1155, + 981, 2156, 981, 955, 1651, 91, 981, 1462, 17, 91, + 23, 1197, 20, 959, 1079, 2269, 2270, 2271, 1981, 39, + 1321, 1200, 2221, 1296, 38, 1161, 0, 972, 2166, 1604, + 1605, 2640, 89, 90, 35, 1793, 89, 90, 2155, 1225, + 2135, 0, 2137, 1274, 0, 0, 89, 90, 2331, 46, + 0, 895, 0, 897, 26, 899, 1982, 2602, 0, 2638, + 0, 972, 1122, 2602, 89, 90, 2661, 1127, 1087, 2101, + 0, 999, 748, 0, 2217, 0, 89, 90, 2654, 0, + 0, 2476, 0, 0, 749, 2288, 0, 0, 102, 979, + 110, 0, 805, 0, 0, 1817, 1655, 810, 2558, 742, + 5, 0, 2562, 0, 0, 5, 9, 11, 13, 14, + 1707, 15, 16, 80, 1814, 2973, 11, 1709, 5, 5, + 767, 16, 5, 3, 1924, 5, 13, 14, 5, 5, + 5, 1053, 1054, 5, 9, 63, 13, 14, 5, 55, + 5, 5, 982, 5, 9, 5, 1811, 5, 1070, 2299, + 5, 13, 14, 13, 14, 5, 1133, 45, 2295, 5, + 2959, 5, 172, 2306, 107, 5, 5, 5, 5, 5, + 2300, 13, 14, 119, 139, 45, 2428, 2976, 63, 147, + 74, 13, 14, 122, 37, 4, 3, 4, 5, 42, + 9, 82, 9, 161, 171, 1106, 1086, 9, 166, 3140, + 1241, 100, 93, 5, 100, 878, 190, 11, 2814, 1250, + 1138, 15, 16, 4, 2357, 2358, 82, 2360, 9, 972, + 177, 122, 1133, 11, 878, 147, 180, 93, 16, 11, + 290, 1033, 1209, 15, 16, 1212, 1213, 295, 53, 802, + 1957, 63, 46, 124, 177, 873, 244, 3, 101, 217, + 104, 244, 63, 1970, 289, 171, 11, 287, 46, 295, + 15, 16, 2795, 192, 2797, 2932, 90, 2521, 275, 237, + 983, 119, 379, 212, 168, 1431, 80, 107, 34, 35, + 3304, 171, 995, 240, 117, 13, 14, 850, 132, 312, + 13, 46, 80, 312, 107, 217, 19, 973, 1209, 1189, + 117, 1212, 1213, 106, 1515, 129, 1462, 240, 31, 1950, + 1951, 1952, 1185, 1186, 132, 159, 2411, 2860, 286, 172, + 183, 342, 45, 46, 30, 80, 180, 295, 120, 64, + 1057, 1133, 38, 30, 30, 1608, 108, 165, 2572, 74, + 40, 38, 38, 366, 454, 172, 1073, 406, 1304, 122, + 365, 192, 161, 11, 388, 208, 313, 163, 41, 2968, + 381, 176, 1280, 1894, 137, 3337, 2124, 120, 3167, 3472, + 2979, 224, 2571, 295, 33, 476, 2915, 370, 193, 4, + 75, 234, 108, 198, 9, 108, 2520, 272, 2522, 1548, + 289, 2970, 249, 289, 1925, 416, 509, 148, 1557, 500, + 59, 218, 194, 516, 514, 1336, 1208, 3431, 2546, 468, + 3513, 518, 3270, 370, 170, 268, 172, 274, 272, 191, + 377, 236, 80, 240, 230, 2568, 355, 2570, 237, 357, + 274, 279, 1343, 1344, 328, 279, 2468, 370, 166, 2673, + 284, 194, 272, 126, 451, 466, 279, 250, 199, 40, + 448, 428, 163, 278, 514, 448, 1521, 260, 3430, 358, + 518, 314, 358, 420, 3405, 422, 284, 3000, 321, 1742, + 2722, 3138, 357, 416, 337, 175, 444, 279, 3084, 514, + 514, 308, 518, 307, 514, 8, 509, 420, 11, 422, + 509, 448, 15, 16, 1296, 323, 19, 20, 21, 514, + 454, 386, 1149, 366, 514, 240, 2101, 472, 361, 3367, + 456, 352, 451, 2650, 1187, 448, 514, 2667, 213, 1974, + 534, 514, 444, 403, 404, 413, 503, 380, 1736, 1092, + 514, 588, 1422, 1187, 1162, 357, 2499, 428, 428, 516, + 2683, 2671, 1320, 413, 512, 512, 357, 1325, 516, 516, + 451, 445, 1853, 1331, 327, 3098, 588, 370, 1480, 1481, + 271, 514, 428, 1436, 2139, 392, 1797, 2367, 393, 3368, + 2165, 425, 2706, 454, 3117, 2172, 391, 476, 3154, 1452, + 476, 2173, 1829, 1505, 175, 1338, 265, 463, 503, 149, + 518, 435, 476, 132, 478, 3140, 449, 421, 514, 514, + 2159, 3140, 446, 507, 508, 509, 510, 2329, 2308, 514, + 513, 478, 1653, 1886, 514, 510, 515, 522, 517, 515, + 517, 517, 522, 478, 1868, 1898, 512, 514, 514, 3089, + 516, 514, 2831, 500, 3094, 2838, 2301, 514, 514, 514, + 200, 320, 514, 1565, 1566, 500, 117, 514, 3043, 514, + 514, 1562, 514, 289, 514, 1928, 514, 1547, 1548, 514, + 3269, 1337, 1935, 1574, 514, 1576, 800, 1557, 514, 1597, + 1598, 1599, 514, 1338, 514, 514, 514, 514, 514, 2962, + 749, 1571, 514, 3278, 1327, 476, 505, 506, 505, 506, + 396, 1602, 749, 505, 506, 2812, 518, 2951, 1971, 396, + 396, 3277, 1975, 507, 508, 509, 510, 2424, 464, 500, + 1600, 30, 108, 1515, 505, 506, 1429, 3381, 161, 190, + 748, 748, 510, 505, 506, 507, 508, 509, 510, 30, + 2003, 800, 789, 161, 2697, 274, 789, 356, 166, 161, + 3070, 749, 3072, 800, 166, 284, 415, 800, 1979, 430, + 505, 506, 507, 508, 509, 510, 883, 800, 71, 72, + 1487, 454, 848, 244, 789, 244, 848, 386, 291, 503, + 3379, 2412, 2413, 2414, 2415, 800, 789, 82, 835, 836, + 1507, 1937, 516, 416, 453, 3364, 2818, 800, 93, 1766, + 258, 259, 0, 26, 237, 1706, 1707, 1838, 479, 1776, + 478, 858, 1779, 274, 3468, 2731, 1608, 517, 2734, 237, + 2736, 2774, 784, 132, 2957, 237, 1543, 2865, 1974, 868, + 868, 514, 500, 1761, 2125, 2873, 868, 464, 551, 868, + 177, 132, 3230, 853, 3232, 868, 837, 838, 244, 840, + 868, 868, 1620, 286, 3423, 902, 903, 1410, 856, 906, + 907, 868, 518, 873, 868, 1766, 868, 868, 286, 2617, + 3405, 2619, 859, 3193, 286, 1776, 3405, 37, 1779, 82, + 3271, 513, 1006, 2468, 13, 14, 1794, 1795, 1796, 1013, + 93, 26, 524, 3492, 244, 1945, 226, 32, 1819, 370, + 1734, 370, 1869, 240, 868, 274, 2052, 3180, 244, 1851, + 59, 3100, 959, 1855, 137, 5, 1858, 375, 376, 868, + 3308, 2486, 868, 868, 1860, 972, 925, 1761, 868, 119, + 868, 978, 979, 108, 1869, 3080, 868, 984, 868, 173, + 987, 988, 272, 990, 991, 992, 993, 1781, 868, 4, + 1742, 868, 1786, 868, 9, 973, 973, 868, 868, 1006, + 868, 868, 2389, 1006, 868, 868, 1013, 226, 1869, 868, + 1013, 868, 868, 1006, 370, 3366, 513, 448, 3369, 448, + 1013, 365, 366, 202, 1031, 1032, 1033, 524, 501, 502, + 503, 1006, 505, 506, 507, 508, 509, 510, 1013, 509, + 173, 352, 137, 1006, 1772, 1052, 516, 2244, 338, 1777, + 1013, 245, 172, 272, 274, 2141, 206, 2602, 2144, 279, + 370, 26, 171, 514, 1071, 1876, 4, 32, 1087, 1880, + 743, 9, 1883, 377, 370, 1082, 1083, 1084, 2786, 1086, + 1087, 2994, 1089, 514, 11, 514, 202, 2310, 208, 2195, + 4, 2272, 448, 1933, 147, 9, 3245, 377, 512, 389, + 514, 30, 516, 26, 224, 251, 342, 1089, 161, 32, + 293, 1118, 245, 166, 234, 3466, 336, 2253, 3322, 46, + 26, 478, 272, 420, 274, 422, 32, 1988, 1135, 1136, + 13, 14, 1168, 1169, 1886, 1171, 1168, 1169, 448, 1171, + 3330, 3331, 416, 500, 515, 381, 1898, 518, 342, 1009, + 512, 448, 448, 80, 516, 1015, 85, 514, 514, 478, + 433, 451, 2131, 1170, 217, 94, 1869, 1174, 1175, 374, + 389, 354, 137, 356, 2014, 377, 1928, 1184, 1185, 1186, + 416, 500, 1189, 1935, 237, 466, 859, 381, 2317, 118, + 132, 370, 1162, 3383, 2323, 514, 514, 132, 293, 25, + 26, 1208, 381, 386, 514, 314, 315, 316, 8, 342, + 1421, 11, 1423, 1424, 137, 15, 16, 159, 514, 1971, + 2443, 512, 416, 1975, 159, 516, 823, 824, 825, 1981, + 466, 137, 451, 286, 1241, 6, 512, 416, 166, 10, + 516, 361, 295, 1250, 13, 14, 46, 18, 381, 2089, + 420, 2003, 422, 53, 1224, 350, 420, 1004, 422, 354, + 380, 190, 33, 13, 14, 1014, 37, 1274, 2136, 1018, + 379, 514, 466, 2818, 203, 1022, 420, 2138, 422, 1026, + 80, 837, 838, 416, 840, 13, 14, 466, 114, 1296, + 514, 386, 26, 26, 507, 2700, 1295, 1295, 32, 32, + 514, 188, 189, 1295, 514, 1312, 1295, 416, 514, 74, + 2171, 2172, 515, 988, 1321, 518, 515, 13, 14, 518, + 2426, 26, 2224, 2429, 2226, 171, 515, 32, 293, 518, + 1312, 1338, 274, 466, 512, 3484, 514, 279, 515, 514, + 449, 518, 284, 515, 279, 145, 518, 515, 1355, 284, + 518, 460, 117, 515, 1361, 420, 518, 422, 1442, 1337, + 1337, 1445, 457, 2227, 515, 2229, 1450, 518, 500, 1453, + 293, 258, 259, 1355, 1458, 515, 176, 2200, 518, 2602, + 1338, 3469, 2568, 3471, 2570, 350, 514, 293, 2585, 354, + 3435, 444, 2298, 193, 515, 2584, 515, 518, 198, 518, + 514, 1408, 1409, 137, 137, 514, 3451, 1414, 875, 1416, + 877, 2569, 514, 1416, 1421, 1422, 1423, 1424, 13, 14, + 514, 386, 2616, 515, 3512, 515, 518, 350, 518, 1436, + 1437, 354, 137, 171, 515, 1442, 236, 518, 1445, 1442, + 514, 1416, 1445, 1450, 2633, 1452, 1453, 1450, 354, 1442, + 1453, 1458, 1445, 1416, 2583, 1458, 2585, 1450, 2473, 2474, + 1453, 3506, 497, 386, 223, 1458, 3511, 1442, 515, 515, + 1445, 518, 518, 402, 516, 1450, 405, 82, 1453, 1442, + 386, 1154, 1445, 1458, 13, 14, 289, 1450, 375, 376, + 1453, 291, 457, 435, 497, 1458, 516, 2451, 2452, 2453, + 435, 13, 14, 108, 446, 42, 170, 2476, 1515, 274, + 515, 446, 518, 518, 279, 3223, 37, 3225, 13, 14, + 2451, 2452, 2451, 515, 515, 2450, 518, 518, 515, 515, + 515, 518, 518, 518, 457, 8, 515, 177, 11, 518, + 1547, 1548, 15, 16, 13, 14, 19, 20, 21, 1556, + 1557, 457, 515, 13, 14, 518, 5, 1564, 2310, 293, + 293, 1544, 202, 328, 1571, 514, 1544, 1544, 173, 515, + 107, 515, 109, 294, 111, 515, 515, 1544, 518, 344, + 1544, 515, 1544, 1544, 518, 171, 2447, 2448, 293, 1596, + 514, 391, 59, 1600, 497, 3140, 1603, 202, 13, 14, + 240, 1608, 1609, 1610, 1611, 1612, 1613, 1614, 1615, 1616, + 1617, 2339, 13, 14, 1621, 1622, 350, 350, 417, 1626, + 354, 354, 515, 1630, 152, 518, 1633, 1634, 1635, 1636, + 1637, 1638, 1639, 1640, 1641, 1617, 3233, 1644, 515, 223, + 245, 518, 2345, 350, 1651, 350, 1653, 13, 14, 354, + 147, 172, 386, 386, 294, 1328, 1692, 1330, 2881, 2537, + 1692, 13, 14, 515, 161, 1672, 518, 515, 177, 166, + 518, 152, 515, 515, 2832, 518, 518, 126, 127, 40, + 445, 386, 2879, 515, 13, 14, 518, 208, 1695, 454, + 152, 2443, 515, 202, 515, 518, 152, 518, 498, 1706, + 1707, 13, 14, 224, 152, 505, 506, 507, 508, 509, + 510, 515, 1670, 234, 13, 14, 13, 14, 13, 14, + 217, 416, 171, 457, 457, 198, 13, 14, 13, 14, + 370, 240, 171, 1408, 1409, 1742, 514, 2560, 274, 1414, + 237, 2602, 13, 14, 1751, 89, 1753, 2499, 221, 13, + 14, 40, 457, 466, 2915, 2623, 365, 366, 2664, 365, + 366, 2955, 152, 2644, 480, 370, 365, 366, 152, 1751, + 516, 1753, 262, 263, 2676, 152, 381, 375, 376, 3433, + 420, 152, 422, 3437, 289, 294, 1793, 459, 460, 286, + 1797, 3446, 3447, 1800, 1801, 352, 2787, 3184, 295, 439, + 3478, 3479, 1135, 1136, 428, 445, 514, 171, 448, 515, + 514, 416, 514, 514, 418, 3038, 219, 224, 291, 299, + 514, 514, 224, 40, 224, 296, 235, 514, 5, 5, + 514, 1838, 324, 514, 439, 514, 514, 5, 5, 3493, + 361, 514, 5, 514, 5, 148, 1853, 9, 514, 514, + 2602, 477, 104, 1860, 1861, 301, 518, 518, 515, 380, + 3405, 466, 1869, 219, 386, 314, 315, 316, 40, 286, + 1834, 2732, 166, 166, 1958, 314, 315, 316, 284, 1886, + 1964, 59, 235, 1890, 1891, 514, 1893, 428, 93, 514, + 518, 1898, 1899, 1900, 1901, 1902, 1903, 1904, 428, 59, + 1907, 1908, 1909, 1910, 1911, 1912, 1913, 1914, 1915, 1916, + 59, 420, 428, 422, 265, 1922, 1923, 3140, 428, 1926, + 108, 1928, 524, 221, 428, 428, 1933, 476, 1935, 152, + 379, 100, 377, 37, 274, 274, 445, 514, 42, 448, + 379, 390, 40, 2870, 274, 2697, 274, 444, 1955, 514, + 3337, 1958, 274, 1960, 198, 1958, 782, 1964, 784, 152, + 171, 1964, 3151, 412, 1971, 1958, 516, 416, 1975, 13, + 1977, 1964, 1979, 515, 1981, 515, 171, 416, 224, 518, + 2882, 2883, 515, 1958, 515, 473, 224, 514, 814, 1964, + 515, 515, 515, 3380, 514, 1958, 2003, 101, 535, 2727, + 449, 1964, 281, 540, 281, 518, 543, 2014, 2015, 2870, + 449, 460, 514, 839, 2875, 512, 516, 516, 515, 516, + 463, 460, 2774, 514, 39, 514, 9, 476, 501, 502, + 503, 514, 505, 506, 507, 508, 509, 510, 3029, 514, + 3009, 472, 514, 3430, 514, 426, 426, 2054, 11, 513, + 352, 500, 2059, 2060, 426, 518, 524, 518, 523, 514, + 279, 2922, 2923, 171, 1737, 514, 428, 514, 172, 180, + 2089, 162, 171, 3267, 1747, 514, 1749, 2084, 2085, 1752, + 518, 8, 2089, 515, 454, 1758, 217, 1760, 15, 16, + 518, 117, 19, 20, 21, 265, 2103, 389, 226, 2106, + 1773, 2108, 3208, 290, 208, 1778, 312, 518, 312, 1782, + 1783, 1784, 1785, 518, 1787, 1788, 180, 2124, 2125, 219, + 224, 126, 127, 2851, 2131, 226, 515, 2134, 514, 2881, + 234, 2134, 274, 226, 333, 287, 295, 40, 514, 466, + 514, 2134, 152, 152, 2151, 171, 972, 3, 2151, 152, + 476, 152, 152, 274, 514, 981, 289, 2164, 2151, 2134, + 3, 40, 289, 2915, 268, 59, 171, 171, 40, 11, + 166, 2134, 515, 515, 2181, 2182, 2151, 515, 515, 514, + 514, 180, 3405, 514, 166, 289, 3, 39, 2151, 3, + 515, 2198, 512, 2200, 512, 303, 428, 428, 428, 428, + 513, 515, 2209, 518, 1030, 515, 314, 315, 316, 516, + 314, 3408, 3118, 497, 523, 515, 497, 321, 171, 515, + 2227, 2228, 2229, 497, 1899, 1900, 1901, 1902, 1903, 1904, + 428, 514, 1907, 1908, 1909, 1910, 1911, 1912, 1913, 1914, + 1915, 1916, 2994, 515, 514, 2227, 2228, 2229, 515, 515, + 515, 514, 514, 155, 791, 249, 514, 361, 473, 2266, + 40, 59, 518, 503, 499, 2272, 1033, 3128, 3129, 290, + 1096, 379, 290, 518, 243, 59, 380, 303, 451, 3140, + 1106, 59, 428, 274, 265, 2304, 3038, 2990, 314, 315, + 316, 152, 514, 202, 2293, 2293, 428, 152, 1124, 152, + 514, 2293, 428, 2310, 2293, 428, 428, 1133, 416, 314, + 315, 316, 515, 281, 281, 40, 352, 514, 518, 515, + 476, 2328, 287, 289, 2331, 40, 177, 152, 279, 515, + 171, 514, 514, 870, 515, 515, 515, 59, 2345, 185, + 166, 449, 3210, 80, 2351, 449, 512, 143, 198, 2331, + 454, 202, 460, 379, 3194, 171, 3196, 3235, 515, 8, + 515, 515, 11, 514, 291, 515, 15, 16, 476, 2351, + 19, 20, 21, 514, 379, 515, 524, 171, 300, 518, + 917, 515, 2389, 3206, 358, 390, 2389, 36, 3140, 240, + 416, 290, 500, 180, 514, 932, 933, 934, 935, 515, + 152, 518, 515, 175, 514, 439, 514, 412, 515, 2084, + 2085, 416, 515, 515, 2389, 40, 40, 514, 86, 3209, + 516, 2428, 3212, 449, 514, 2428, 2389, 3396, 515, 518, + 514, 454, 518, 3339, 460, 171, 2443, 2514, 514, 198, + 515, 1208, 475, 294, 449, 515, 515, 513, 985, 460, + 476, 513, 2459, 518, 515, 460, 515, 2476, 2465, 2466, + 3348, 3329, 313, 515, 515, 289, 503, 59, 515, 2476, + 515, 476, 515, 2146, 500, 515, 204, 117, 476, 40, + 2487, 226, 514, 2490, 88, 2492, 191, 279, 514, 279, + 2509, 40, 2499, 2500, 516, 500, 2503, 2504, 40, 516, + 3323, 2508, 2509, 516, 516, 513, 503, 428, 2515, 514, + 516, 60, 274, 107, 516, 476, 40, 9, 428, 370, + 314, 315, 316, 2530, 7, 8, 377, 516, 1354, 1296, + 13, 516, 1358, 2540, 289, 515, 19, 516, 516, 516, + 23, 2560, 25, 26, 3405, 2528, 516, 30, 31, 32, + 2528, 2528, 35, 2560, 516, 38, 39, 106, 516, 42, + 516, 2528, 45, 46, 2528, 515, 2528, 2528, 516, 420, + 2534, 422, 221, 516, 501, 502, 503, 516, 505, 506, + 507, 508, 509, 510, 516, 379, 516, 516, 439, 513, + 516, 351, 2599, 516, 445, 2602, 2603, 448, 2605, 516, + 516, 516, 516, 516, 516, 516, 89, 90, 516, 515, + 2617, 518, 2619, 514, 514, 335, 514, 2599, 514, 102, + 515, 2603, 416, 2605, 107, 108, 109, 110, 111, 514, + 518, 59, 515, 198, 191, 513, 518, 459, 91, 515, + 344, 514, 291, 40, 2643, 2643, 152, 2320, 516, 3377, + 515, 2643, 201, 3405, 2643, 449, 40, 1194, 124, 152, + 40, 515, 40, 366, 366, 26, 460, 515, 40, 514, + 1207, 32, 454, 309, 514, 2639, 60, 514, 514, 40, + 518, 279, 476, 190, 248, 454, 439, 2694, 74, 292, + 2697, 1228, 74, 514, 9, 80, 515, 515, 368, 60, + 2719, 250, 2666, 514, 513, 2669, 500, 2714, 2715, 515, + 2794, 260, 2719, 513, 59, 2722, 93, 133, 503, 2722, + 514, 272, 106, 272, 40, 439, 289, 515, 514, 459, + 292, 514, 8, 292, 515, 11, 515, 204, 515, 15, + 16, 289, 2749, 289, 515, 106, 122, 386, 1515, 40, + 451, 365, 148, 25, 1291, 304, 2763, 36, 365, 297, + 1297, 2768, 2769, 868, 2875, 2586, 2773, 2774, 2222, 60, + 46, 2778, 1033, 0, 2781, 2782, 137, 53, 1759, 2786, + 2787, 2599, 1671, 2790, 2265, 1803, 2292, 2794, 3301, 2808, + 1232, 2794, 2351, 20, 2801, 783, 23, 2990, 3354, 3461, + 3111, 2794, 3409, 2635, 80, 3418, 3454, 356, 1195, 3169, + 2215, 38, 2228, 2655, 2578, 106, 26, 201, 3407, 2794, + 47, 3416, 32, 2212, 2605, 3404, 1292, 2539, 2503, 2504, + 40, 2794, 381, 1320, 2334, 1661, 1860, 1338, 2845, 3299, + 201, 1608, 3237, 2538, 3380, 3333, 2853, 3483, 2528, 2198, + 60, 400, 501, 502, 503, 2164, 505, 506, 507, 508, + 509, 510, 89, 90, 91, 2872, 250, 1005, 1154, 145, + 1005, 1731, 1358, 1178, 2881, 102, 260, 2410, 3389, 2181, + 1695, 3310, 3198, 1730, 175, 23, 1177, 2151, 272, 250, + 2563, 2564, 1180, 2722, 1969, 3033, 106, 789, 983, 260, + 176, 2389, 1439, 800, 2388, 2435, 3132, 1984, 2915, 981, + 201, 272, 1869, 981, 981, 3314, 3313, 193, 2591, 2061, + 304, 981, 198, 981, 981, 981, 981, 137, 2466, 2016, + 2165, 2107, 293, 2606, 2607, 2608, 2609, 2610, 2611, 2612, + 2613, 2614, 2615, 304, 2104, 2063, 2483, 1208, 1670, 2801, + 1977, 102, 1437, -1, 2961, 2962, -1, -1, -1, 250, + 236, -1, -1, 1789, -1, -1, 515, -1, -1, 260, + -1, -1, 356, -1, -1, 1742, 1802, 1803, -1, -1, + 2962, 272, -1, 2990, -1, -1, 2993, 2994, -1, 350, + 3009, 201, -1, 354, -1, 356, -1, 381, -1, -1, + -1, -1, 3009, 40, -1, -1, -1, 2971, -1, -1, + -1, -1, -1, 304, -1, 291, 400, -1, -1, 2694, + 381, -1, 3029, 60, -1, 386, -1, -1, -1, -1, + -1, 3038, -1, -1, -1, 1296, 3043, -1, -1, 400, + 250, -1, -1, 1869, -1, -1, -1, 3054, 3055, 1875, + 260, 3058, 535, 3060, -1, -1, -1, 540, -1, -1, + 543, -1, 272, -1, -1, 356, -1, -1, 551, 106, + 107, -1, -1, -1, -1, -1, -1, -1, 3085, -1, + 117, -1, -1, 293, -1, -1, -1, -1, -1, -1, + 381, -1, -1, -1, 304, -1, 457, 8, -1, -1, + 11, -1, 3109, -1, 15, 16, -1, -1, 1033, 400, + -1, -1, -1, -1, -1, 391, -1, -1, -1, 1886, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, 446, 1365, 1366, -1, 147, -1, 106, 1371, -1, - 1373, -1, 457, 1259, -1, -1, -1, -1, 1836, 161, - -1, -1, -1, 1386, 166, 1388, 1389, -1, 473, 171, - 1848, -1, -1, 1396, -1, 1398, -1, 1400, 180, -1, - 1403, -1, -1, 185, -1, 1408, -1, -1, 1411, -1, - 1413, -1, 497, 2790, 1417, -1, 1419, -1, 1421, -1, - 1878, -1, -1, -1, 37, -1, 511, 1885, -1, 42, - -1, -1, -1, -1, -1, 217, 175, 2544, -1, 388, - -1, 527, -1, -1, -1, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, 237, -1, -1, -1, -1, - -1, -1, 201, -1, -1, 1923, 879, -1, -1, 1927, - 303, -1, -1, -1, -1, 1933, -1, -1, -1, -1, - 313, 314, 315, -1, -1, -1, 1259, -1, 101, 575, - 576, -1, -1, -1, -1, -1, -1, 1955, -1, -1, - 12, -1, -1, 2255, 286, 17, -1, 289, -1, -1, - -1, 250, -1, 295, -1, -1, -1, 29, -1, -1, - -1, 260, -1, -1, -1, -1, 1529, 2634, -1, -1, - 616, 43, 44, 272, -1, -1, -1, 1540, -1, -1, - -1, 627, -1, 376, -1, 221, 495, -1, -1, 2926, - -1, -1, 334, 502, 503, 504, 505, 506, 507, 172, - -1, -1, -1, -1, -1, 304, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 357, 1579, -1, -1, -1, - 413, -1, -1, -1, -1, -1, -1, -1, 1474, 675, - 102, -1, -1, -1, -1, 208, -1, -1, -1, -1, - -1, -1, -1, -1, 2711, -1, -1, -1, 694, -1, - -1, 224, -1, 446, -1, 291, 355, -1, -1, -1, - -1, 234, -1, 1626, 457, 1628, -1, 1630, -1, -1, - -1, 413, -1, 2385, -1, -1, -1, 1640, 1641, 378, - 473, 8, -1, -1, -1, -1, -1, -1, 15, 16, - 1653, 1064, 19, 20, 21, 268, -1, -1, 397, 441, - 1073, -1, -1, -1, 497, -1, -1, -1, -1, 451, - 2097, -1, -1, -1, -1, 1678, 289, 1680, 511, -1, - -1, 1567, -1, -1, -1, -1, -1, -1, 470, 2441, - 472, 473, 1695, 1696, -1, -1, -1, -1, -1, -1, - 313, 1474, -1, 1706, 1707, 1708, 1709, 320, 1711, -1, - -1, 2818, -1, -1, 1717, 8, 1719, -1, 11, -1, - -1, -1, 15, 16, -1, -1, -1, 509, 1731, 1732, - 512, 513, 514, 1736, 1737, -1, -1, -1, 1741, 1742, - 1743, 1744, -1, 1746, 1747, 2852, -1, 360, -1, -1, - -1, -1, -1, 46, -1, -1, -1, -1, 1761, -1, - 53, -1, -1, -1, 377, -1, 924, 1770, 1771, 1772, - 1773, 1774, -1, -1, -1, -1, -1, -1, 864, -1, - -1, -1, -1, -1, -1, 1788, -1, 80, 8, -1, - -1, 11, 2544, -1, 1567, 15, 16, 2255, -1, 19, - 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 897, -1, -1, -1, 1701, 36, -1, 904, -1, - 2927, -1, 498, 499, 500, -1, 502, 503, 504, 505, - 506, 507, -1, 446, -1, -1, -1, -1, 2265, -1, - -1, -1, -1, -1, -1, -1, 1849, 1850, 1851, 1852, - 1853, 1854, 145, 939, 1857, 1858, 1859, 1860, 1861, 1862, - 1863, 1864, 1865, 1866, 2971, 951, -1, -1, -1, -1, + -1, 1898, 1669, 3140, 1671, 46, -1, -1, 175, -1, + 350, -1, 53, -1, 354, 1682, 356, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 1033, -1, + -1, 1928, -1, -1, 201, -1, -1, -1, 1935, 80, + -1, 381, 3179, 3180, 1033, -1, 386, 3184, 2853, 2852, + -1, 3184, -1, 1720, -1, -1, -1, 3194, -1, 3196, + 400, 3198, 3156, 3157, -1, 3202, -1, 3204, 3180, 3206, + -1, -1, -1, -1, 1971, -1, -1, -1, 1975, 3184, + 37, 3218, -1, 250, 1981, 42, 3223, -1, 3225, -1, + -1, 3184, 498, 260, -1, -1, 3233, -1, -1, 505, + 506, 507, 508, 509, 510, 272, 2003, 274, -1, 3246, + -1, -1, -1, -1, 3251, -1, -1, 457, -1, -1, + -1, -1, -1, -1, 1515, -1, -1, -1, 741, 742, + 743, -1, -1, -1, 3246, 176, -1, 304, -1, -1, + 3234, -1, 3236, -1, 101, -1, 1813, 1814, 1815, 1816, + -1, 1818, 193, 1208, -1, -1, -1, 198, -1, -1, + -1, -1, -1, 3300, -1, -1, -1, -1, -1, 782, + 783, 784, -1, 3310, -1, -1, 789, 534, 791, 1033, + -1, -1, 3276, -1, -1, -1, 3323, 800, 2993, 356, + -1, 804, 805, -1, -1, 236, 809, 810, 3292, -1, + 3337, -1, 3296, 1208, 3337, -1, -1, -1, -1, -1, + 823, 824, 825, -1, 381, 172, -1, 1608, 3355, 1208, + -1, -1, -1, 180, 837, 838, -1, 840, -1, -1, + -1, -1, 3337, 400, 3, 402, 5, -1, 405, -1, + 853, 1296, -1, 3380, 3337, -1, 859, 3380, -1, -1, + 291, 208, -1, -1, -1, -1, -1, 870, -1, 3396, + 873, -1, -1, -1, -1, 0, -1, 224, 3405, -1, + 3407, -1, -1, -1, -1, 3380, -1, 234, 2234, -1, + -1, -1, 1949, -1, -1, -1, -1, 3380, -1, -1, + -1, 1296, -1, 3430, -1, 3407, -1, 3430, -1, 68, + 69, -1, -1, -1, 917, 918, -1, 1296, 3445, 3446, + 3447, 268, -1, -1, -1, -1, -1, 3120, -1, 932, + 933, 934, 935, 936, -1, 3430, -1, -1, 3422, -1, + 2286, -1, 289, -1, -1, 3472, 949, 3430, 3141, 3142, + 109, 110, -1, -1, 113, 114, -1, 514, 2304, -1, + 391, 1742, -1, -1, 1208, 3449, -1, 314, -1, -1, + -1, 37, -1, 3166, 321, 100, 42, -1, -1, 982, + 983, -1, 985, -1, -1, 988, 3513, -1, -1, -1, + -1, 994, 995, -1, -1, -1, -1, 1000, -1, 2345, + -1, 748, 749, 1006, -1, -1, -1, 2353, -1, 2355, + 1013, -1, -1, 2359, 361, 2361, -1, -1, -1, -1, + 1023, -1, 147, 2310, -1, -1, -1, 1030, -1, 188, + 189, -1, 13, 380, -1, 101, 161, 1040, 19, -1, + -1, 166, 789, -1, -1, -1, 171, -1, -1, -1, + 31, -1, 1296, 800, -1, 180, -1, -1, -1, -1, + 185, -1, -1, -1, 45, 46, -1, 498, -1, -1, + 1515, -1, -1, -1, 505, 506, 507, 508, 509, 510, + -1, -1, -1, -1, -1, -1, -1, -1, 1091, -1, + -1, -1, 217, 252, 253, 254, 255, 256, 257, 258, + 259, 848, 449, 262, 263, 1886, 172, 454, -1, 151, + -1, -1, 237, -1, -1, -1, -1, 1898, -1, -1, + 1515, 868, -1, -1, -1, -1, -1, 108, -1, 171, + -1, -1, 8, -1, 1137, 11, 1515, 1140, -1, 15, + 16, -1, 208, 19, 20, 21, -1, 1928, -1, -1, + -1, 1154, 1155, -1, 1935, -1, 2443, -1, 224, 1162, + 36, 286, -1, 1608, 289, -1, -1, -1, 234, -1, + 295, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 1184, -1, -1, -1, -1, -1, -1, -1, -1, + 1971, 1194, 1195, -1, 1975, 55, -1, -1, -1, -1, + 1981, -1, 268, -1, 1207, -1, -1, -1, 3401, -1, + 335, -1, 2499, 1608, -1, -1, 375, 376, -1, -1, + -1, 1224, 2003, 289, -1, 1228, 973, -1, -1, 1608, + 1233, -1, -1, 358, -1, 74, -1, 2294, -1, -1, + -1, 80, 102, -1, -1, -1, -1, -1, 314, -1, + 2307, 2308, 2309, -1, 93, 321, 2602, -1, -1, 1006, + -1, -1, -1, 123, -1, 2322, 1013, -1, 2325, -1, + -1, 1515, 314, 315, 316, -1, 2333, -1, 117, -1, + 119, 141, -1, -1, -1, 145, 1289, -1, 1291, 2635, + -1, 416, -1, -1, 1297, 361, -1, 1742, -1, -1, + -1, 1048, -1, -1, -1, -1, -1, 167, 1311, -1, + 170, 470, 471, 1060, 380, 2602, -1, 1320, -1, 444, + -1, -1, 1325, -1, 1327, 1328, 186, 1330, 1331, 454, + -1, -1, -1, -1, -1, 494, 495, 379, -1, -1, + 1087, -1, -1, -1, -1, 221, -1, 1742, 473, -1, + 475, 476, -1, -1, -1, 1358, -1, -1, -1, -1, + 2417, 2418, 2419, 1742, 1608, -1, -1, 206, -1, -1, + -1, -1, -1, -1, 416, -1, -1, -1, 2724, -1, + -1, -1, -1, 449, -1, -1, 8, 512, -1, 11, + 515, 516, 517, 15, 16, -1, -1, 19, 20, 21, + -1, -1, -1, -1, -1, 1408, 1409, 449, -1, -1, + 2697, 1414, -1, 1416, 36, 291, -1, -1, 460, 279, + -1, 1168, 1169, -1, 1171, -1, 1429, 287, 1431, 1432, + -1, -1, -1, -1, 476, 274, 1439, -1, -1, 1442, + 279, 1886, 1445, -1, -1, -1, -1, 1450, -1, 309, + 1453, -1, 2798, 1898, -1, 1458, -1, 1460, 500, 1462, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 37, -1, 2634, 176, 1887, 42, -1, -1, -1, -1, - -1, -1, 978, -1, -1, -1, 1899, -1, -1, -1, - 193, -1, -1, -1, -1, 198, 1909, -1, -1, -1, - -1, -1, 1915, -1, -1, -1, -1, -1, 1921, -1, - -1, -1, -1, 1926, 291, -1, -1, 2385, 1701, -1, - -1, -1, 1935, 1936, -1, -1, -1, -1, -1, -1, - 1098, -1, -1, 236, 101, -1, -1, -1, -1, -1, - 1836, -1, 1365, 1366, -1, -1, -1, -1, 1371, 2711, - -1, -1, 1848, -1, -1, -1, 3073, -1, -1, -1, - -1, 1057, 1058, -1, 1060, -1, -1, -1, -1, -1, - -1, -1, -1, 2441, -1, -1, -1, -1, -1, -1, - -1, -1, 1878, -1, -1, -1, -1, -1, 291, 1885, - 2003, 221, -1, -1, 2007, -1, -1, -1, -1, 2012, - -1, -1, -1, -1, -1, 172, -1, -1, -1, -1, - -1, -1, 924, -1, -1, -1, -1, -1, -1, -1, - 542, -1, 2035, 2036, -1, -1, -1, 1923, -1, -1, - -1, 1927, -1, -1, -1, -1, -1, 1933, -1, 2052, - -1, 208, -1, -1, -1, -1, -1, -1, -1, 571, - 572, -1, -1, 1836, -1, -1, 2818, 224, -1, 1955, - -1, 291, -1, -1, -1, 1848, -1, 234, 2505, 2506, - -1, -1, 2085, -1, -1, -1, 2544, -1, -1, -1, - -1, -1, -1, -1, 2097, 388, -1, -1, -1, 2102, - 2852, 1259, -1, 2106, -1, 1878, 2533, -1, -1, -1, - -1, 268, 1885, -1, -1, -1, -1, -1, -1, -1, - -1, 2548, 2549, 2550, 2551, 2552, 2553, 2554, 2555, 2556, - 2557, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, -1, 2146, -1, -1, 2149, -1, -1, 2152, - 1923, -1, -1, -1, 1927, -1, 313, -1, -1, -1, - 1933, -1, -1, 320, -1, 2168, -1, -1, -1, -1, - -1, -1, -1, -1, 686, 2927, 2634, -1, -1, -1, - -1, -1, 1955, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 1098, -1, -1, -1, - -1, -1, 495, 360, -1, -1, -1, -1, 1294, 502, - 503, 504, 505, 506, 507, 8, -1, -1, 11, 2971, - 377, -1, 15, 16, -1, 3332, 19, 20, 21, -1, - -1, 2234, -1, -1, -1, -1, -1, 2240, -1, -1, - 1653, -1, -1, 36, -1, -1, -1, -1, 2251, 2252, - 2253, 2254, -1, 2711, -1, -1, -1, -1, -1, -1, - -1, -1, 2265, -1, 2267, -1, -1, 2270, -1, -1, - -1, -1, 2275, -1, -1, -1, 2279, -1, 498, 499, - 500, -1, 502, 503, 504, 505, 506, 507, -1, 446, - -1, -1, -1, -1, 2297, 2298, -1, 2300, -1, -1, - -1, -1, -1, 55, -1, -1, -1, -1, -1, -1, - -1, -1, 1398, -1, 1400, -1, 1474, 1403, -1, -1, - -1, 3073, 1408, -1, -1, 1411, 2329, 1413, -1, -1, - -1, 1417, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 2349, -1, -1, -1, - 102, -1, -1, -1, 2357, 2358, 2359, 1259, -1, -1, - 2818, -1, 2789, -1, 2367, -1, 2369, -1, 2371, 2255, - -1, 123, -1, -1, 2377, -1, -1, -1, -1, 891, - -1, -1, -1, -1, -1, -1, -1, -1, 8, 141, - -1, 11, -1, 145, 2852, 15, 16, 17, 18, 19, - 20, 21, 914, -1, -1, -1, -1, 2410, -1, 1567, - -1, -1, -1, -1, -1, 167, 36, 1503, 170, -1, - -1, -1, -1, -1, -1, -1, 46, -1, 221, -1, - -1, -1, -1, 53, 186, -1, 1849, 1850, 1851, 1852, - 1853, 1854, 2445, 2446, 1857, 1858, 1859, 1860, 1861, 1862, - 1863, 1864, 1865, 1866, -1, -1, -1, -1, -1, -1, - 80, -1, -1, -1, -1, -1, -1, -1, -1, 2927, - 982, 2474, -1, -1, -1, -1, -1, 2480, -1, -1, - -1, -1, 2255, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2496, -1, -1, 1582, 2500, 291, 2385, - -1, 2504, 2505, 2506, -1, -1, -1, 2510, 2511, 2512, - -1, 2514, 8, 2971, -1, 11, -1, -1, -1, 15, - 16, -1, -1, 19, 20, 21, -1, 279, -1, -1, - 2533, -1, 2535, 1045, 1046, 287, -1, -1, -1, -1, - -1, -1, -1, 1701, -1, 2548, 2549, 2550, 2551, 2552, - 2553, 2554, 2555, 2556, 2557, 2441, 176, 309, -1, -1, - -1, -1, -1, -1, 1650, -1, -1, -1, -1, -1, - -1, -1, 1474, 193, -1, -1, 2579, -1, 198, -1, - 3332, -1, 2585, 335, -1, -1, -1, -1, -1, -1, - -1, -1, 2595, -1, 879, -1, -1, -1, -1, -1, - -1, 221, 222, -1, -1, -1, -1, -1, 2611, -1, - -1, -1, 2385, -1, -1, 3073, 236, 2620, -1, -1, - -1, -1, 2035, 2036, -1, -1, 3053, -1, 2631, -1, - -1, -1, -1, -1, 2637, -1, -1, -1, -1, -1, - -1, 2644, 2645, 2646, 2647, -1, -1, 3074, 3075, -1, - -1, -1, -1, 2656, 274, -1, 2659, 277, 2544, -1, - 2663, 2664, -1, -1, -1, 1567, -1, -1, 2441, 2672, - 3097, 291, -1, -1, 294, -1, -1, -1, 1836, -1, + -1, -1, 514, -1, 151, -1, 336, -1, -1, -1, + -1, -1, -1, 1928, -1, -1, -1, 2774, -1, 328, + 1935, 1886, -1, -1, 171, -1, -1, 2554, 1742, -1, + -1, -1, -1, 1898, -1, 344, -1, 1886, -1, 8, + -1, -1, 11, -1, -1, -1, 15, 16, -1, 1898, + 19, 20, 21, -1, -1, -1, 1971, -1, -1, 2310, + 1975, -1, -1, 1928, -1, -1, 1981, -1, -1, -1, + 1935, -1, -1, -1, -1, -1, -1, -1, -1, 1928, + -1, -1, -1, -1, -1, -1, 1935, -1, 2003, -1, + -1, -1, -1, -1, -1, -1, -1, 1570, -1, -1, + 551, -1, -1, -1, -1, -1, 1971, -1, 1581, -1, + 1975, -1, -1, -1, -1, -1, 1981, -1, 2645, -1, + 1337, -1, 1971, -1, 2881, -1, 1975, -1, -1, 221, + -1, -1, 1981, -1, -1, -1, 445, -1, 2003, -1, + -1, -1, -1, -1, -1, 454, -1, 1620, -1, -1, + -1, 2967, -1, -1, 2003, 501, 502, 503, 2915, 505, + 506, 507, 508, 509, 510, -1, -1, 314, 315, 316, + 8, -1, 1886, 11, 2990, -1, -1, 15, 16, -1, + 2707, 2708, 2709, 2710, 1898, -1, -1, -1, -1, -1, + -1, -1, 2443, -1, 1667, -1, 1669, -1, 1671, 291, + -1, -1, -1, -1, -1, -1, -1, -1, 46, 1682, + 1683, -1, -1, -1, 1928, 53, -1, -1, -1, -1, + -1, 1935, 1695, -1, -1, 1442, -1, -1, 1445, 198, + -1, -1, 379, 1450, -1, -1, 1453, 2994, -1, -1, + -1, 1458, 80, -1, -1, -1, -1, 1720, 2499, 1722, + -1, -1, 221, -1, -1, 3071, -1, 1971, -1, -1, + -1, 1975, -1, 1736, 1737, -1, -1, 1981, -1, 416, + -1, -1, -1, -1, 1747, 1748, 1749, 1750, -1, 1752, + -1, 3038, -1, -1, -1, 1758, -1, 1760, -1, 2003, + -1, 742, 743, -1, -1, -1, -1, -1, -1, 1772, + 1773, -1, 449, -1, 1777, 1778, -1, 145, -1, 1782, + 1783, 1784, 1785, 460, 1787, 1788, -1, -1, -1, -1, + -1, -1, 291, -1, 3140, -1, -1, 1544, -1, 476, + -1, 1804, 2859, -1, -1, -1, -1, -1, 176, -1, + 1813, 1814, 1815, 1816, 1817, 1818, -1, -1, -1, -1, + -1, 2602, -1, 500, -1, 193, -1, -1, -1, 1832, + 198, -1, -1, -1, -1, -1, -1, 514, -1, -1, + -1, -1, -1, -1, -1, 1848, -1, -1, -1, -1, + -1, -1, -1, 3140, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2310, -1, -1, 236, -1, + -1, -1, -1, -1, -1, -1, 1623, -1, 859, 501, + 502, 503, -1, 505, 506, 507, 508, 509, 510, -1, + -1, -1, -1, -1, -1, -1, 1899, 1900, 1901, 1902, + 1903, 1904, -1, -1, 1907, 1908, 1909, 1910, 1911, 1912, + 1913, 1914, 1915, 1916, -1, 2310, 2697, -1, -1, -1, + -1, -1, 3268, 291, -1, -1, -1, -1, -1, -1, + -1, 2310, -1, -1, 1937, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 1692, 1949, -1, -1, -1, + -1, -1, -1, -1, -1, 1958, -1, -1, -1, -1, + -1, 1964, 8, -1, -1, 11, 1969, -1, -1, 15, + 16, 1974, -1, 19, 20, 21, -1, -1, -1, -1, + 1983, 1984, -1, -1, -1, -1, -1, -1, -1, -1, + 36, -1, -1, 2774, -1, -1, -1, -1, 2443, -1, + -1, -1, 501, 502, 503, -1, 505, 506, 507, 508, + 509, 510, -1, -1, -1, -1, -1, -1, -1, 1000, + -1, -1, -1, 391, -1, 8, -1, -1, 11, -1, + -1, -1, 15, 16, -1, -1, -1, -1, -1, -1, + -1, -1, 1023, -1, -1, -1, -1, -1, 2443, 2052, + -1, -1, -1, 2056, 2499, -1, -1, -1, 2061, 3405, + -1, -1, -1, 46, 2443, -1, 2310, -1, -1, -1, + 53, -1, -1, -1, -1, -1, -1, -1, 1825, -1, + -1, 2084, 2085, -1, -1, -1, -1, 1834, -1, -1, + -1, -1, -1, -1, -1, -1, 3442, 80, 2101, -1, + 2881, -1, -1, 3160, 2499, -1, -1, -1, -1, -1, + 1091, -1, -1, -1, -1, -1, -1, -1, 3405, -1, + 2499, -1, -1, -1, 3181, -1, -1, -1, -1, -1, + 498, 2134, -1, -1, 2915, -1, -1, 505, 506, 507, + 508, 509, 510, 2146, -1, -1, -1, -1, 2151, -1, + -1, -1, 2155, -1, -1, -1, -1, 2602, -1, -1, + -1, -1, 145, 8, -1, -1, 11, -1, -1, -1, + 15, 16, -1, 1154, 1155, 221, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 3244, -1, -1, + -1, -1, 2195, 176, -1, 2198, -1, -1, 2201, 2443, + -1, 46, -1, -1, -1, -1, -1, 2602, 53, -1, + 193, 1958, -1, 2994, 2217, 198, -1, 1964, 3275, -1, + -1, -1, -1, 2602, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 80, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 291, -1, -1, -1, -1, + -1, -1, 2697, 236, -1, 2499, -1, 3038, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 1848, -1, -1, -1, -1, 1781, 2699, -1, -1, -1, - -1, -1, -1, -1, 1790, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, -1, -1, -1, -1, -1, - 1878, -1, -1, -1, -1, 221, 2729, 1885, 2731, -1, - -1, -1, -1, -1, -1, -1, 2149, -1, -1, -1, - -1, -1, 2745, -1, -1, -1, 2749, -1, 2634, -1, - -1, -1, 2755, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 1923, 2769, -1, 388, 1927, - -1, 2544, 2775, 1285, 1286, 1933, 1288, -1, -1, -1, - -1, -1, -1, -1, -1, 2788, 2789, 2790, 1073, -1, - -1, -1, -1, 2796, -1, 291, -1, 1955, -1, 1701, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 2813, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 1909, -1, 2711, -1, -1, -1, 1915, + -1, -1, -1, -1, -1, 2288, -1, -1, -1, -1, + -1, 2294, -1, -1, -1, -1, -1, -1, -1, -1, + 145, -1, 2697, 2306, 2307, 2308, 2309, -1, 291, -1, + -1, -1, -1, -1, -1, -1, -1, 2320, 2697, 2322, + -1, -1, 2325, -1, -1, -1, 2329, -1, -1, 2774, + 2333, 176, -1, -1, -1, -1, 2339, -1, -1, -1, + -1, -1, 2089, -1, -1, -1, 1327, 1328, 193, 1330, + -1, -1, -1, 198, 2357, 2358, -1, 2360, 2602, 3140, + 2107, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 2123, -1, -1, 2774, + -1, -1, -1, -1, -1, 8, 2389, 2134, 11, -1, + -1, 236, 15, 16, -1, 2774, 19, 20, 21, -1, + -1, -1, -1, -1, 2151, -1, 2409, -1, 391, -1, + -1, -1, -1, -1, 2417, 2418, 2419, -1, -1, -1, + -1, -1, -1, 2426, -1, 2428, 2429, -1, -1, -1, + -1, -1, 2435, -1, -1, -1, 2881, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 291, -1, -1, -1, + 1431, -1, -1, 2697, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, 2468, -1, -1, -1, -1, + 2915, -1, -1, -1, -1, -1, -1, -1, -1, 1460, + -1, 1462, -1, -1, -1, -1, 2881, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 8, -1, -1, - 11, -1, -1, -1, 15, 16, -1, -1, 19, 20, - 21, 2634, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3332, 495, 1388, -1, 498, 499, - 500, -1, 502, 503, 504, 505, 506, 507, -1, -1, - 2893, -1, -1, -1, -1, 515, -1, -1, -1, -1, - -1, 3328, 2905, -1, -1, -1, -1, 1419, -1, 1421, - -1, -1, -1, -1, -1, -1, 2919, -1, -1, -1, - -1, -1, -1, 2926, -1, -1, -1, -1, -1, -1, - -1, -1, 2818, -1, 1836, -1, -1, -1, 2711, -1, - -1, -1, -1, -1, -1, -1, 1848, -1, -1, 2952, - -1, -1, -1, -1, 2040, 2958, -1, -1, -1, -1, - -1, -1, -1, 2966, -1, -1, 2852, -1, -1, -1, - -1, -1, 2058, -1, -1, -1, 1878, -1, -1, -1, - -1, -1, -1, 1885, -1, -1, -1, -1, 2074, -1, - -1, -1, -1, -1, -1, -1, -1, 3000, -1, 2085, - -1, -1, 498, 499, 500, -1, 502, 503, 504, 505, - 506, 507, -1, -1, -1, -1, 2102, -1, -1, 3022, - -1, 1923, -1, -1, 3027, 1927, -1, -1, -1, -1, - -1, 1933, 2445, 2446, -1, -1, -1, -1, -1, -1, - -1, 2927, -1, -1, -1, 2818, -1, -1, 662, 663, - 3053, -1, -1, 1955, -1, -1, -1, -1, -1, -1, - 221, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 3074, 3075, -1, -1, -1, -1, -1, -1, 2852, - 1365, 1366, -1, -1, -1, 2971, 1371, -1, 3091, -1, - 3093, -1, -1, -1, 3097, -1, -1, 2255, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 3111, -1, - -1, 3114, -1, -1, 728, -1, -1, -1, 732, -1, + 2503, 2504, 2881, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 498, -1, -1, -1, -1, + 2915, -1, 505, 506, 507, 508, 509, 510, -1, 2532, + 2774, -1, -1, -1, -1, 2538, 2915, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 391, -1, -1, 2994, + -1, 2554, -1, -1, -1, 2558, -1, 2304, -1, 2562, + 2563, 2564, -1, -1, -1, 2568, 2569, 2570, -1, 2572, + -1, -1, -1, 0, -1, -1, -1, -1, -1, -1, + 988, -1, -1, -1, -1, -1, -1, -1, 2591, -1, + 2593, -1, -1, 3038, -1, -1, -1, -1, 221, 2994, + -1, -1, -1, 2606, 2607, 2608, 2609, 2610, 2611, 2612, + 2613, 2614, 2615, -1, -1, 2994, -1, -1, -1, -1, + -1, -1, -1, -1, 3405, -1, -1, -1, -1, -1, + -1, -1, -1, 8, -1, 2638, 11, 2881, -1, -1, + 15, 16, 2645, 3038, 19, 20, 21, -1, -1, -1, + -1, -1, 2655, 498, -1, -1, -1, -1, -1, 3038, + 505, 506, 507, 508, 509, 510, -1, -1, 291, -1, + 2673, 2915, -1, 100, 988, -1, -1, -1, -1, -1, + 2683, 98, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 2694, -1, -1, -1, 3140, -1, 2700, -1, -1, + -1, -1, -1, -1, 2707, 2708, 2709, 2710, -1, 126, + 127, -1, -1, -1, -1, -1, 2719, -1, -1, 2722, + 147, -1, -1, 2726, 2727, -1, -1, -1, -1, 2476, + -1, -1, 2735, -1, 161, -1, -1, -1, -1, 166, + -1, 1722, -1, -1, 171, 3140, -1, -1, -1, -1, + 2994, -1, -1, 180, 171, -1, 1737, -1, 185, 2762, + -1, 3140, 2509, -1, -1, -1, 1747, 1175, 1749, -1, + -1, 1752, -1, -1, -1, -1, 1184, 1758, -1, 1760, + -1, 2528, -1, -1, -1, -1, -1, 2534, -1, 2792, + 217, 2794, 1773, -1, 3038, -1, -1, 1778, -1, -1, + -1, 1782, 1783, 1784, 1785, 2808, 1787, 1788, -1, 2812, + 237, -1, -1, 2560, -1, 2818, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2832, + -1, -1, -1, -1, -1, 2838, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 221, -1, 2851, 2852, + 2853, -1, -1, -1, -1, -1, 2859, -1, -1, 286, + -1, -1, 289, -1, -1, -1, -1, -1, 295, -1, + 1184, -1, -1, 2876, -1, -1, -1, -1, 501, 502, + 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, + -1, -1, 2639, -1, -1, -1, 3140, 314, 315, 316, + -1, -1, -1, -1, -1, -1, -1, -1, 335, -1, + -1, -1, -1, -1, -1, -1, 291, -1, -1, 2666, + -1, -1, 2669, -1, -1, -1, -1, -1, -1, -1, + -1, 358, -1, -1, -1, -1, -1, -1, -1, 8, + -1, -1, 11, -1, -1, -1, 15, 16, -1, -1, + -1, 2954, -1, -1, 2957, -1, 1937, -1, -1, -1, + 3405, -1, 379, -1, -1, -1, -1, 2970, -1, -1, + -1, -1, 2719, 390, -1, -1, -1, 46, -1, -1, + -1, -1, -1, 2986, 53, -1, -1, -1, 1969, 416, + 2993, -1, -1, 1974, -1, 412, -1, -1, -1, 416, + 1408, 1409, -1, -1, -1, -1, 1414, -1, -1, 0, + 3405, 80, -1, -1, -1, -1, 3019, 444, -1, 436, + -1, -1, 3025, -1, -1, -1, 3405, 454, -1, -1, + 3033, -1, 449, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 460, -1, -1, 473, 2794, 475, 476, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 476, + -1, 2808, -1, -1, 3067, -1, -1, -1, -1, -1, + -1, 2052, -1, -1, -1, -1, 145, -1, -1, -1, + 2061, -1, -1, 500, -1, 512, 3089, -1, 515, 516, + 517, 3094, -1, -1, 1408, 1409, -1, 514, -1, -1, + 1414, -1, -1, -1, -1, -1, -1, 176, -1, 100, + -1, -1, -1, -1, -1, -1, -1, 3120, -1, -1, + -1, 835, 836, -1, 193, -1, 501, 502, 503, 198, + 505, 506, 507, 508, 509, 510, -1, -1, 3141, 3142, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 291, 3134, -1, -1, -1, 3138, 3139, -1, -1, 3142, + -1, -1, -1, -1, -1, -1, 147, 3160, -1, 3162, + -1, 3405, -1, 3166, -1, 2146, -1, 236, -1, -1, + 161, -1, -1, -1, -1, 166, -1, -1, 3181, -1, + 171, 3184, -1, -1, -1, -1, -1, -1, 902, 180, + -1, -1, 906, 907, 185, -1, -1, -1, -1, -1, + -1, 3204, -1, -1, -1, 3208, 3209, -1, -1, 3212, + -1, -1, -1, -1, 2195, -1, -1, -1, -1, -1, + 2201, -1, 291, -1, 2971, -1, 217, -1, -1, -1, + -1, -1, -1, -1, 3237, -1, -1, -1, -1, -1, + -1, 3244, -1, -1, -1, 959, 237, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 2927, -1, -1, -1, 772, -1, - -1, -1, -1, 2249, 3167, -1, -1, -1, 1680, -1, - -1, 3174, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 1696, -1, -1, 3073, -1, -1, - -1, -1, -1, -1, 1706, -1, 1708, -1, 2971, 1711, - -1, 3204, -1, 8, -1, 1717, 11, 1719, -1, -1, - 15, 16, -1, -1, -1, -1, -1, -1, 2631, -1, - 1732, -1, -1, -1, -1, 1737, -1, 2385, -1, 1741, - 1742, 1743, 1744, -1, 1746, 1747, -1, -1, -1, 3242, - -1, 46, -1, 2656, -1, -1, -1, -1, 53, -1, - 8, -1, -1, 11, -1, 869, -1, 15, 16, -1, - -1, 875, 3265, -1, 878, -1, -1, 881, 882, 883, - 884, -1, -1, -1, -1, 80, -1, -1, -1, -1, - -1, -1, -1, 2441, -1, -1, -1, 2700, 46, 3292, - -1, -1, -1, -1, -1, 53, -1, -1, -1, -1, - 3073, 3304, 3305, -1, 3307, 3308, -1, -1, 922, 923, - -1, -1, -1, 3316, -1, -1, -1, -1, -1, -1, - -1, 0, 80, 1608, -1, 3328, -1, -1, -1, 943, - -1, -1, 2418, -1, -1, -1, -1, 498, 499, 500, - 145, 502, 503, 504, 505, 506, 507, 3350, 962, -1, - -1, -1, -1, 2255, 3357, -1, -1, -1, -1, 973, - 974, 975, -1, 977, 978, 2451, -1, -1, 1653, -1, - -1, 176, -1, -1, -1, 1887, -1, 2790, -1, -1, - -1, -1, -1, -1, 2470, -1, 2544, 145, 193, -1, - 2476, -1, 3395, 198, -1, 1009, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 3410, -1, 1921, - -1, -1, 1026, 1027, 1926, 3, 2502, 5, 176, -1, - -1, 100, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 236, -1, -1, -1, 193, -1, -1, -1, -1, - 198, -1, -1, -1, -1, 1059, 3332, -1, -1, 1063, - 1064, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 1074, 1075, -1, -1, -1, -1, -1, -1, 147, -1, - -1, -1, -1, -1, -1, -1, 2634, -1, 236, -1, - 68, 69, 161, 2385, -1, -1, 291, 166, -1, -1, - -1, 2003, 171, -1, 2580, -1, -1, -1, -1, -1, - 2012, 180, -1, -1, -1, -1, 185, -1, -1, -1, - -1, -1, -1, 2926, -1, -1, -1, -1, -1, -1, - 2606, 109, 110, 2609, -1, 113, 114, -1, -1, -1, - -1, -1, -1, 291, -1, -1, -1, -1, 217, 2441, + -1, -1, 3009, -1, 978, -1, -1, -1, -1, -1, + 984, -1, 3275, 987, -1, -1, 990, 991, 992, 993, + -1, -1, -1, -1, -1, -1, -1, 1695, -1, -1, + -1, -1, -1, -1, -1, 286, -1, -1, 289, -1, + -1, -1, -1, -1, 295, -1, -1, -1, -1, -1, + -1, 3314, -1, -1, -1, -1, -1, 1031, 1032, -1, + -1, -1, 391, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 3337, 1649, -1, -1, 1052, 2320, + -1, -1, -1, -1, 335, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 1071, -1, -1, + -1, 3364, -1, -1, -1, -1, -1, 358, 1082, 1083, + 1084, -1, 1086, 1087, 3377, 3378, -1, 3380, 3381, -1, + -1, 1695, -1, -1, -1, -1, 3389, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 3401, -1, + -1, -1, -1, -1, 1118, -1, -1, -1, -1, 3156, + 3157, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 3423, 1135, 1136, -1, -1, 416, -1, 3430, -1, 498, + -1, -1, -1, -1, -1, -1, 505, 506, 507, 508, + 509, 510, -1, -1, -1, 2426, -1, -1, 2429, -1, + -1, -1, -1, 444, 2435, -1, 1170, -1, -1, -1, + 1174, 1175, -1, 454, -1, 3468, -1, -1, -1, -1, + -1, 1185, 1186, -1, -1, -1, -1, -1, -1, -1, + 3483, -1, 473, -1, 475, 476, -1, 3234, -1, 3236, + -1, 1899, 1900, 1901, 1902, 1903, 1904, -1, -1, 1907, + 1908, 1909, 1910, 1911, 1912, 1913, 1914, 1915, 1916, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2711, -1, -1, -1, -1, 237, 3332, - -1, -1, -1, -1, 1849, 1850, 1851, 1852, 1853, 1854, - 2656, -1, 1857, 1858, 1859, 1860, 1861, 1862, 1863, 1864, - 1865, 1866, -1, 388, -1, 2097, -1, -1, -1, -1, - -1, -1, -1, -1, 1208, -1, -1, -1, -1, -1, - 188, 189, -1, 1217, -1, -1, -1, 286, -1, -1, - 289, -1, -1, -1, -1, -1, 295, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 1241, -1, -1, - 388, -1, -1, -1, 2146, -1, -1, -1, -1, -1, - 2152, -1, 2544, -1, -1, 2731, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 334, -1, -1, -1, 2745, - 2818, -1, -1, -1, 252, 253, 254, 255, 256, 257, - 258, 259, -1, -1, 262, 263, -1, -1, 357, -1, + -1, 512, -1, -1, 515, 516, 517, 1241, -1, 3276, + -1, -1, -1, -1, -1, -1, 1250, -1, -1, -1, + -1, -1, -1, -1, -1, 3292, -1, -1, -1, 3296, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 495, -1, -1, -1, 2852, -1, -1, 502, 503, 504, - 505, 506, 507, -1, 1318, -1, -1, -1, -1, -1, + 1274, -1, 68, 69, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 2558, -1, -1, + -1, 2562, 2563, 2564, -1, 1899, 1900, 1901, 1902, 1903, + 1904, -1, -1, 1907, 1908, 1909, 1910, 1911, 1912, 1913, + 1914, 1915, 1916, 109, 110, -1, -1, 113, 114, -1, + 2591, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2606, 2607, 2608, 2609, 2610, + 2611, 2612, 2613, 2614, 2615, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 1361, -1, -1, + -1, -1, -1, -1, -1, -1, 24, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 3134, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 2634, -1, 413, -1, -1, 495, -1, -1, - -1, -1, -1, -1, 502, 503, 504, 505, 506, 507, - 2035, 2036, -1, 2265, -1, -1, -1, -1, -1, -1, - -1, -1, 441, -1, 1378, -1, 1380, 1381, -1, 2927, - -1, -1, 451, -1, -1, -1, -1, -1, -1, 1393, - 1394, -1, -1, -1, -1, 373, 374, -1, -1, -1, - -1, 470, -1, 472, 473, -1, 1410, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 2711, - -1, -1, -1, 2971, -1, -1, -1, -1, -1, -1, - 2906, 68, 69, -1, -1, -1, -1, -1, -1, -1, - 509, -1, -1, 512, 513, 514, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 3422, 2084, 2085, -1, -1, + -1, -1, 188, 189, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 2367, 2942, -1, -1, 2371, - -1, -1, 109, 110, 2149, 2377, 113, 114, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 467, - 468, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 3449, -1, -1, -1, -1, 1421, -1, 1423, + 1424, -1, -1, 81, -1, -1, -1, -1, -1, 2700, + -1, -1, 1436, 1437, -1, -1, -1, -1, -1, 97, + -1, -1, -1, -1, -1, -1, -1, -1, 1452, -1, + -1, -1, -1, -1, -1, 2726, 252, 253, 254, 255, + 256, 257, 258, 259, -1, -1, 262, 263, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 1515, -1, 491, 492, -1, -1, -1, -1, 1523, - -1, -1, -1, -1, -1, 3073, 2818, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 8, -1, -1, - 11, -1, -1, -1, 15, 16, -1, -1, -1, -1, - -1, 188, 189, -1, -1, -1, -1, -1, 1562, -1, - 2852, -1, -1, -1, 1568, 1569, 1570, 1571, 1572, 1573, - 1574, 1575, -1, -1, -1, 46, 1580, 1581, -1, -1, - -1, 1585, 53, -1, -1, 1589, -1, -1, 1592, 1593, - 1594, 1595, 1596, 1597, 1598, 1599, 1600, -1, 2500, 1603, - -1, -1, 2504, 2505, 2506, -1, 1610, -1, 1612, 80, - -1, -1, 3088, 3089, -1, 252, 253, 254, 255, 256, - 257, 258, 259, -1, -1, 262, 263, 1631, -1, -1, - -1, 2533, -1, -1, -1, 2927, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 2548, 2549, 2550, 2551, - 2552, 2553, 2554, 2555, 2556, 2557, -1, -1, -1, -1, - 1664, 1665, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 145, -1, -1, -1, -1, 2971, - -1, -1, -1, -1, -1, -1, -1, -1, 3164, -1, - 3166, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 176, -1, -1, -1, -1, + 2084, 2085, -1, -1, -1, -1, -1, -1, 146, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 156, -1, + 2198, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 168, -1, -1, -1, -1, 173, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 193, -1, -1, -1, -1, 198, -1, 3205, - -1, -1, -1, -1, -1, 2637, 373, 374, -1, -1, - -1, -1, -1, -1, -1, 3221, -1, -1, 1752, 3225, - -1, -1, 1756, -1, -1, 1759, 1760, -1, -1, -1, - -1, 2663, -1, -1, -1, 236, -1, -1, -1, -1, - 2445, 2446, -1, -1, -1, -1, -1, 0, -1, -1, - -1, 3073, -1, -1, 3332, -1, -1, -1, -1, -1, - 1794, -1, 0, -1, -1, -1, -1, -1, -1, 22, - -1, -1, -1, -1, -1, -1, 1810, 1811, -1, 32, - -1, 34, 35, -1, -1, -1, -1, -1, -1, -1, - 291, -1, -1, -1, 47, -1, -1, -1, -1, 52, - 467, 468, -1, -1, -1, -1, 1840, 1841, 61, 1843, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 75, -1, 491, 492, -1, -1, -1, -1, - 879, -1, -1, 86, -1, -1, -1, -1, 1872, 1873, - -1, -1, 1876, 3349, 511, 98, -1, 100, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 2789, 111, -1, - -1, -1, 100, -1, -1, -1, -1, -1, -1, -1, - 3376, 1905, -1, -1, 127, 128, -1, 1911, -1, -1, - -1, -1, -1, -1, 137, -1, -1, 388, -1, -1, - 143, -1, -1, -1, -1, 1929, -1, 1931, 151, -1, - 153, 154, -1, -1, -1, -1, -1, -1, -1, 147, - -1, -1, -1, -1, 167, -1, -1, -1, 171, -1, - -1, -1, -1, 161, -1, -1, 2631, -1, 166, -1, - -1, -1, 1966, 171, -1, -1, -1, -1, -1, -1, - -1, -1, 180, -1, 197, -1, -1, 185, -1, 998, - -1, 2656, -1, -1, -1, -1, -1, -1, -1, -1, - 213, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 2005, -1, -1, -1, -1, 2010, 2011, -1, 217, - -1, -1, -1, -1, -1, -1, 239, -1, -1, -1, - -1, -1, 1041, -1, 495, -1, -1, -1, -1, 237, - -1, 502, 503, 504, 505, 506, 507, -1, -1, -1, - 3332, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 2054, -1, -1, 2057, 1073, 2059, 2958, -1, -1, -1, - -1, -1, -1, -1, 2966, -1, -1, -1, -1, -1, - -1, 2075, -1, -1, -1, -1, -1, -1, 286, 22, - -1, 289, -1, -1, -1, -1, -1, 295, -1, 32, - 313, 314, 315, -1, -1, -1, -1, -1, 321, -1, - -1, 324, -1, -1, -1, -1, 1125, -1, -1, 52, - -1, 2115, -1, -1, -1, 2790, -1, -1, 61, -1, - 3022, -1, -1, -1, -1, 3027, 334, -1, 2132, 2133, - 353, -1, 75, -1, -1, -1, -1, -1, -1, 362, - -1, -1, -1, 86, -1, -1, -1, 2151, -1, 357, - -1, 3053, -1, 376, -1, 98, 2160, 100, -1, -1, - 383, -1, -1, -1, 387, -1, -1, -1, 111, -1, - -1, -1, 3074, 3075, 397, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 127, -1, 409, -1, -1, -1, - 413, -1, -1, -1, 137, 3097, -1, -1, -1, -1, - 143, -1, -1, -1, -1, 413, -1, -1, 151, -1, - 433, -1, -1, 2217, -1, -1, -1, -1, -1, 2223, - -1, -1, -1, 446, 167, 1244, 449, -1, 171, 452, - -1, -1, -1, 441, 457, -1, 3138, -1, -1, -1, - -1, -1, -1, 451, -1, -1, -1, -1, -1, -1, - 473, 2926, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 470, -1, 472, 473, -1, -1, -1, 2273, - 213, -1, -1, 1292, 497, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 511, -1, - -1, 514, -1, -1, -1, -1, 239, 1316, -1, -1, - -1, 509, -1, -1, 512, 513, 514, -1, -1, -1, - -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, + -1, -1, -1, -1, 202, -1, -1, -1, -1, -1, + -1, -1, 1556, -1, -1, -1, -1, -1, -1, -1, + 1564, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 375, + 376, 2852, -1, -1, -1, -1, -1, 245, -1, -1, + -1, 249, -1, -1, 2198, -1, -1, -1, -1, 1603, + -1, -1, -1, -1, -1, 1609, 1610, 1611, 1612, 1613, + 1614, 1615, 1616, -1, -1, -1, -1, 1621, 1622, -1, + -1, -1, 1626, -1, -1, -1, 1630, -1, -1, 1633, + 1634, 1635, 1636, 1637, 1638, 1639, 1640, 1641, -1, -1, + 1644, -1, -1, -1, -1, 8, -1, 1651, 11, 1653, + -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, + -1, 319, -1, -1, -1, -1, -1, -1, 1672, -1, + -1, -1, -1, 36, 470, 471, 334, -1, -1, -1, + -1, -1, -1, 46, -1, -1, -1, -1, -1, -1, + 53, -1, -1, -1, -1, -1, -1, -1, 494, 495, + -1, -1, 1706, 1707, -1, -1, -1, -1, -1, 367, + -1, -1, 370, -1, -1, -1, -1, 80, 514, -1, + -1, -1, 8, 381, -1, 11, 384, -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + -1, -1, -1, -1, 402, -1, -1, -1, -1, -1, + 36, -1, -1, -1, 3025, -1, -1, -1, 416, -1, + 46, -1, 3033, 988, -1, 423, -1, 53, -1, -1, + -1, -1, -1, -1, -1, 433, -1, -1, -1, -1, + -1, 439, -1, -1, -1, -1, -1, -1, -1, 1793, + -1, -1, -1, 1797, 80, -1, 1800, 1801, -1, -1, + -1, -1, -1, -1, -1, 2503, 2504, -1, 466, -1, + -1, -1, -1, 176, -1, -1, -1, -1, 3089, -1, + -1, -1, -1, 3094, -1, -1, -1, -1, -1, -1, + 193, -1, -1, -1, 1838, 198, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 3120, + -1, -1, -1, -1, -1, -1, 1860, 1861, 221, 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 36, -1, 1361, -1, 1363, -1, 1365, 1366, -1, 1368, - 46, -1, 1371, -1, -1, 1374, -1, 53, 1377, -1, - -1, -1, -1, 1382, -1, -1, 1385, -1, -1, -1, - 313, 314, 315, -1, -1, -1, -1, -1, 321, -1, - -1, 324, -1, -1, 80, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 2401, -1, -1, - -1, -1, -1, 2407, 2408, -1, -1, -1, 1427, -1, - 353, -1, -1, -1, 2418, -1, -1, -1, -1, 362, - -1, -1, -1, -1, -1, 2429, 3328, -1, 2432, -1, - 2434, -1, -1, 376, -1, -1, -1, -1, 2442, -1, - 383, -1, -1, -1, 387, -1, 2450, 2451, -1, -1, - -1, -1, -1, 2457, 397, -1, -1, -1, -1, 3134, - -1, -1, -1, -1, -1, -1, 409, -1, 2472, -1, - 413, -1, -1, -1, -1, -1, -1, -1, 2482, -1, - 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 433, -1, 1511, -1, -1, -1, -1, 193, 2502, -1, - -1, -1, 198, 446, -1, -1, -1, -1, 1527, 452, - -1, -1, -1, -1, 457, -1, -1, -1, 1537, 1538, - 1539, -1, -1, -1, 1543, 221, 222, -1, 1547, -1, - 473, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 236, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 497, 2559, -1, 2561, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 511, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 274, -1, - -1, 277, -1, -1, -1, -1, -1, -1, -1, -1, - 1609, -1, -1, -1, -1, 291, 3, -1, 294, -1, - -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, 1647, 36, - -1, -1, -1, 40, -1, -1, -1, -1, -1, 46, - -1, -1, -1, -1, 1663, -1, 53, 2651, 2652, 1668, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 1685, -1, -1, -1, - -1, -1, -1, 80, -1, -1, -1, -1, -1, -1, - -1, -1, 2686, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 388, -1, -1, -1, 2700, -1, -1, -1, - -1, 2705, 2706, -1, -1, -1, 2710, -1, -1, -1, - -1, 2715, -1, -1, 2718, 2719, -1, -1, -1, 2723, - 2724, -1, -1, 2727, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 2738, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, - -1, -1, 15, 16, -1, -1, 19, 20, 21, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 176, - -1, -1, -1, -1, -1, -1, -1, -1, 2782, -1, - -1, -1, -1, 46, -1, -1, 193, -1, -1, -1, - 53, 198, -1, -1, -1, -1, -1, -1, -1, 495, - -1, -1, 498, 499, 500, 2809, 502, 503, 504, 505, - 506, 507, -1, -1, 221, 222, -1, 80, -1, 1838, - -1, -1, -1, -1, -1, 521, -1, 1846, 1847, 236, - 1849, 1850, 1851, 1852, 1853, 1854, -1, -1, 1857, 1858, - 1859, 1860, 1861, 1862, 1863, 1864, 1865, 1866, 1867, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 274, -1, -1, - 277, -1, -1, 68, 69, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 291, -1, -1, 294, 8, -1, - -1, 11, -1, 2897, -1, 15, 16, 17, 18, 19, - 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 176, 109, 110, 36, -1, 113, 114, - 40, -1, -1, -1, -1, -1, 46, -1, -1, -1, - 193, -1, -1, 53, -1, 198, -1, -1, 2942, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 1969, -1, -1, 1972, -1, -1, -1, -1, 2962, -1, - 80, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 2976, 236, -1, -1, -1, -1, -1, -1, - -1, 388, -1, 2987, 2988, -1, -1, 2991, -1, 2993, - -1, -1, -1, 188, 189, -1, -1, -1, -1, 2018, - -1, -1, -1, 2022, 2023, 2024, 2025, 2026, 2027, 2028, - 2029, -1, -1, -1, 3018, -1, 2035, 2036, -1, 2038, - 2039, -1, -1, -1, -1, -1, -1, -1, 291, -1, - -1, 2050, -1, -1, 2053, -1, -1, -1, 3042, -1, - -1, -1, 2061, 2062, 2063, 2064, 2065, 2066, 2067, 2068, - 2069, 2070, -1, -1, -1, -1, 176, 252, 253, 254, - 255, 256, 257, 258, 259, -1, -1, 262, 263, -1, - -1, -1, -1, 193, -1, -1, -1, 2096, 198, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, 221, 222, -1, -1, -1, 3110, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 236, -1, -1, -1, - -1, -1, -1, -1, 3128, 388, -1, -1, 3132, -1, - 2149, -1, 3136, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3148, -1, -1, -1, -1, 3153, - -1, 3155, -1, -1, 274, -1, -1, 277, -1, 3163, + 3141, 3142, -1, 236, -1, -1, -1, -1, -1, -1, + -1, -1, 1107, -1, -1, -1, 1890, 1891, -1, 1893, + 176, -1, -1, -1, -1, 3166, -1, -1, -1, 2503, + 2504, -1, -1, -1, -1, -1, -1, 193, -1, -1, + -1, 274, 198, -1, 277, -1, -1, -1, 1922, 1923, + 0, -1, 1926, -1, -1, 1150, -1, -1, 291, -1, + -1, 294, -1, -1, -1, 221, 222, 3208, -1, -1, + -1, -1, 22, -1, -1, -1, -1, -1, -1, -1, + 236, 1955, 32, -1, 34, 35, 1960, -1, -1, 1184, + -1, -1, -1, -1, -1, -1, -1, 47, -1, -1, + -1, -1, 52, 1977, -1, 1979, -1, -1, -1, -1, + -1, 61, -1, -1, -1, -1, -1, -1, 274, -1, + -1, 277, -1, -1, -1, 75, 2694, -1, -1, -1, + -1, -1, -1, -1, 84, 291, 86, -1, 294, -1, + 1235, 2015, -1, -1, -1, -1, -1, -1, 98, -1, + 100, 2719, -1, -1, -1, -1, -1, -1, 391, -1, + -1, 111, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 126, 127, 128, -1, + 2054, -1, -1, -1, 1279, 2059, 2060, 137, -1, 1284, + -1, -1, -1, 143, -1, 2763, -1, -1, -1, -1, + -1, 151, -1, 153, 154, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 167, -1, -1, + 2694, 171, -1, -1, -1, -1, -1, -1, -1, 2103, + -1, -1, 2106, -1, 2108, 391, -1, -1, -1, 1334, + -1, -1, -1, -1, -1, 2719, -1, 197, -1, -1, + 2124, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 3401, -1, -1, 213, 1359, 498, -1, -1, 501, 502, + 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, + -1, -1, -1, -1, -1, 2853, -1, -1, -1, 239, + 2164, 524, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 2181, 2182, 1404, + -1, 1406, -1, 1408, 1409, -1, 1411, -1, -1, 1414, + -1, -1, 1417, -1, -1, 1420, 2200, -1, -1, -1, + 1425, -1, -1, 1428, -1, 2209, -1, -1, -1, -1, + -1, -1, 498, -1, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, -1, -1, -1, -1, + -1, -1, 518, -1, 314, 315, 316, -1, -1, -1, + -1, -1, 322, 1468, -1, 325, -1, -1, -1, 2853, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 291, -1, -1, 294, -1, -1, 3181, 373, 374, + -1, -1, 2266, -1, -1, -1, -1, -1, 2272, -1, + -1, -1, -1, -1, 354, -1, -1, -1, -1, -1, + -1, -1, -1, 363, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2993, -1, -1, -1, 379, + -1, -1, -1, -1, -1, -1, 386, -1, -1, -1, + 390, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 400, -1, -1, -1, 2328, -1, -1, 1552, -1, -1, + -1, -1, 412, -1, -1, -1, 416, -1, -1, -1, + -1, -1, -1, 1568, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 1578, 1579, 1580, 436, -1, -1, 1584, + -1, -1, -1, 1588, -1, -1, -1, -1, -1, 449, + -1, -1, 452, -1, -1, 455, -1, -1, -1, -1, + 460, -1, -1, -1, -1, -1, -1, -1, -1, 2993, + -1, -1, -1, -1, -1, -1, 476, -1, -1, 68, + 69, -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, 2218, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 2228, - 2229, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3228, -1, -1, -1, -1, -1, - -1, -1, 495, -1, 3238, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, -1, -1, 3251, -1, -1, + 500, -1, -1, -1, -1, 1650, -1, -1, -1, -1, + -1, -1, -1, -1, 514, -1, -1, 517, -1, -1, + 109, 110, -1, -1, 113, 114, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 2459, -1, -1, -1, -1, + -1, 2465, 2466, -1, 1689, -1, -1, -1, -1, -1, + -1, -1, 2476, -1, -1, -1, -1, -1, -1, -1, + 1705, -1, -1, 2487, -1, 1710, 2490, -1, 2492, -1, + -1, -1, -1, -1, -1, -1, 2500, -1, -1, -1, + -1, -1, 1727, -1, 2508, 2509, 3204, -1, -1, -1, + -1, 2515, -1, -1, -1, -1, -1, -1, -1, 188, + 189, -1, -1, -1, -1, -1, 2530, -1, -1, 3, + -1, -1, -1, -1, 8, -1, 2540, 11, -1, -1, + -1, 15, 16, 17, 18, 19, 20, 21, -1, -1, + -1, -1, -1, -1, -1, -1, 2560, -1, -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, 252, 253, 254, 255, 256, 257, 258, + 259, -1, -1, 262, 263, -1, -1, -1, -1, -1, + 3204, -1, -1, -1, -1, -1, 80, -1, -1, -1, + -1, -1, -1, 2617, -1, 2619, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 2282, -1, -1, -1, -1, 388, -1, - -1, -1, 467, 468, -1, -1, -1, -1, -1, 3283, - -1, -1, -1, -1, -1, 2304, 2305, 2306, -1, -1, - 2309, 2310, 2311, 2312, 2313, 2314, 491, 492, -1, 2318, - 2319, 2320, 2321, 2322, 2323, 2324, 2325, 2326, 2327, -1, - -1, -1, -1, 2332, 2333, -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, 2360, -1, -1, 36, -1, -1, 2366, -1, -1, - 24, -1, -1, -1, 46, -1, -1, -1, -1, -1, - -1, 53, -1, -1, -1, -1, -1, -1, 3372, 3373, - 3374, -1, -1, -1, -1, 495, -1, -1, 498, 499, - 500, 2400, 502, 503, 504, 505, 506, 507, 80, -1, - -1, -1, 512, -1, -1, 3399, -1, -1, -1, -1, - 2419, -1, -1, 2422, 2423, -1, -1, 81, -1, -1, - -1, 2430, 2431, -1, -1, 26, -1, -1, -1, -1, - -1, 32, -1, 97, -1, 2444, 2445, 2446, 2447, 40, - 2449, -1, -1, -1, 2453, -1, 3440, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 60, - -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, -1, -1, -1, -1, - -1, -1, 146, -1, -1, -1, -1, -1, -1, 36, - -1, -1, 156, -1, 176, -1, -1, -1, 2507, 46, - -1, -1, -1, -1, 168, 106, 53, -1, -1, 173, - -1, 193, -1, -1, -1, -1, 198, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 80, -1, -1, 137, -1, 202, 221, - 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 236, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, - -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, - -1, 245, -1, -1, -1, 249, -1, -1, -1, -1, - -1, -1, 274, 36, -1, 277, -1, -1, -1, -1, - 201, 2610, -1, 46, -1, -1, -1, -1, -1, 291, - 53, 26, 294, -1, -1, -1, -1, 32, -1, -1, - -1, -1, 2631, -1, -1, 40, -1, -1, -1, 176, - -1, -1, -1, -1, -1, -1, -1, 80, -1, -1, - -1, -1, -1, -1, -1, 60, 193, -1, -1, 250, - -1, 198, -1, -1, 318, -1, -1, -1, -1, 260, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 333, - -1, 272, -1, -1, 221, 222, -1, -1, 2687, -1, - 2689, -1, -1, -1, -1, -1, 2695, -1, -1, 236, - -1, 106, 293, -1, -1, 2704, -1, -1, 2707, -1, - 2709, -1, 366, 304, 2713, 369, 388, 2716, 2717, -1, - -1, 2720, 2721, -1, 378, -1, -1, 381, -1, 2728, - -1, -1, 137, -1, -1, -1, -1, 274, 2737, -1, - 277, -1, -1, 176, -1, 399, -1, -1, -1, -1, - -1, -1, -1, 2752, 291, -1, -1, 294, 349, 413, - 193, -1, 353, -1, 355, 198, 420, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 430, -1, -1, -1, - -1, -1, 436, -1, -1, -1, -1, 378, 221, 222, - -1, 2790, 383, -1, -1, -1, 201, -1, -1, -1, - -1, -1, -1, 236, -1, -1, 397, -1, -1, 463, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 495, -1, -1, 498, 499, 500, -1, - 502, 503, 504, 505, 506, 507, -1, -1, -1, -1, - 512, 274, -1, -1, 277, 250, -1, -1, -1, -1, - -1, 388, -1, -1, -1, 260, -1, -1, 291, -1, - -1, 294, -1, 454, -1, -1, -1, 272, -1, -1, + -1, -1, -1, 1888, -1, -1, -1, -1, -1, -1, + -1, 1896, 1897, -1, 1899, 1900, 1901, 1902, 1903, 1904, + -1, -1, 1907, 1908, 1909, 1910, 1911, 1912, 1913, 1914, + 1915, 1916, 1917, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 176, -1, -1, -1, 375, 376, -1, -1, + 2714, 2715, -1, -1, -1, -1, -1, -1, -1, 193, + -1, -1, -1, -1, 198, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 293, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 304, + -1, -1, -1, -1, -1, 2749, -1, 221, 222, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2763, + -1, -1, 236, -1, 2768, 2769, -1, -1, -1, 2773, + -1, -1, -1, -1, 2778, -1, -1, 2781, 2782, -1, + -1, -1, 2786, 2787, -1, -1, 2790, -1, -1, -1, + -1, -1, -1, 2018, -1, -1, 2021, 2801, -1, -1, + 274, 470, 471, 277, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 291, -1, -1, + 294, -1, -1, -1, -1, 494, 495, 8, -1, -1, + 11, -1, -1, -1, 15, 16, -1, -1, 19, 20, + 21, 2845, 2067, -1, -1, -1, 2071, 2072, 2073, 2074, + 2075, 2076, 2077, 2078, -1, 36, -1, -1, -1, 2084, + 2085, -1, 2087, 2088, -1, 46, -1, -1, 2872, -1, + -1, -1, 53, -1, 2099, -1, -1, 2102, -1, -1, + -1, -1, -1, -1, -1, 2110, 2111, 2112, 2113, 2114, + 2115, 2116, 2117, 2118, 2119, -1, -1, -1, -1, 80, + 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, + 18, 19, 20, 21, -1, -1, -1, 391, -1, -1, + 2145, -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, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 2961, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 80, 2198, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 2925, 2926, -1, -1, + -1, -1, -1, -1, -1, 176, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 3009, -1, -1, -1, -1, + -1, -1, 193, -1, -1, -1, -1, 198, -1, -1, + -1, -1, -1, -1, 498, 3029, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -1, -1, 3043, + 221, 222, 2267, -1, -1, -1, -1, -1, -1, -1, + 3054, 3055, 2277, 2278, 3058, 236, 3060, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 349, -1, -1, -1, 353, -1, - 355, 2950, 2951, -1, -1, 388, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, -1, 378, -1, 512, 2975, -1, 383, -1, - 2979, -1, 2981, 2982, 2983, -1, -1, 2986, -1, -1, - 2989, 2990, 397, -1, -1, -1, -1, -1, -1, 2998, + -1, 3085, -1, -1, -1, 193, -1, -1, -1, -1, + 198, -1, -1, 274, -1, -1, 277, -1, -1, -1, + -1, -1, -1, -1, -1, 3109, -1, -1, -1, -1, + 291, -1, -1, 221, 222, -1, -1, 2342, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 236, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 2364, + 2365, 2366, -1, -1, 2369, 2370, 2371, 2372, 2373, 2374, + -1, -1, -1, 2378, 2379, 2380, 2381, 2382, 2383, 2384, + 2385, 2386, 2387, -1, -1, -1, 274, 2392, 2393, 277, + -1, -1, -1, -1, -1, 3179, -1, -1, -1, -1, + -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, 3198, 2420, 8, -1, 3202, 11, + 2425, -1, 3206, 15, 16, 17, 18, 19, 20, 21, + 391, -1, -1, -1, 3218, -1, -1, -1, -1, 3223, + -1, 3225, -1, -1, 36, -1, -1, -1, 24, 3233, + -1, -1, -1, 2458, 46, -1, -1, -1, -1, -1, + -1, 53, -1, -1, -1, -1, -1, 3251, -1, -1, + -1, -1, 2477, -1, -1, 2480, 2481, -1, -1, -1, + -1, -1, -1, 2488, 2489, -1, -1, -1, 80, -1, + -1, -1, -1, -1, -1, -1, -1, 2502, 2503, 2504, + 2505, -1, 2507, 391, -1, 81, 2511, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 3300, -1, -1, -1, + -1, 97, -1, -1, -1, -1, 3310, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 498, -1, 3323, + 501, 502, 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 2565, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 146, 3355, -1, -1, -1, -1, -1, -1, -1, -1, + 156, -1, -1, -1, 176, -1, -1, -1, -1, -1, + -1, -1, 168, -1, -1, -1, -1, 173, -1, -1, + -1, 193, -1, -1, -1, -1, 198, -1, -1, -1, + 498, -1, -1, 501, 502, 503, -1, 505, 506, 507, + 508, 509, 510, -1, -1, -1, 202, 515, -1, 221, + 222, -1, -1, -1, -1, -1, -1, -1, 8, -1, + -1, 11, -1, -1, 236, 15, 16, 17, 18, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 3445, 3446, 3447, -1, 2670, 36, -1, -1, 245, + -1, -1, -1, 249, -1, -1, 46, -1, -1, -1, + -1, -1, 274, 53, -1, 277, -1, -1, 3472, 2694, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 291, + -1, -1, 294, -1, -1, -1, -1, -1, -1, -1, + 80, -1, 8, -1, -1, 11, -1, -1, -1, 15, + 16, 17, 18, 19, 20, 21, -1, -1, -1, 3513, + -1, -1, -1, -1, -1, -1, -1, 313, -1, -1, + 36, -1, -1, 319, -1, 2750, -1, 2752, -1, -1, + 46, -1, -1, 2758, -1, -1, -1, 53, 334, -1, + -1, -1, 2767, -1, -1, 2770, -1, 2772, -1, -1, + -1, 2776, -1, -1, 2779, 2780, -1, -1, 2783, 2784, + -1, -1, -1, -1, 80, -1, 2791, -1, -1, -1, + -1, 367, -1, -1, 370, 2800, -1, -1, -1, 391, + -1, -1, -1, -1, 8, 381, 176, 11, 384, -1, + 2815, 15, 16, 17, 18, 19, 20, 21, -1, -1, + -1, -1, -1, 193, -1, -1, 402, -1, 198, -1, + -1, -1, 36, -1, -1, -1, -1, -1, -1, -1, + 416, -1, 46, -1, -1, -1, 422, 423, 2853, 53, + -1, 221, 222, -1, -1, -1, -1, 433, -1, -1, + -1, -1, -1, 439, -1, -1, 236, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 80, -1, -1, -1, + 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 466, -1, -1, -1, -1, -1, -1, 193, -1, -1, + -1, -1, 198, -1, 274, -1, 498, 277, -1, 501, + 502, 503, -1, 505, 506, 507, 508, 509, 510, -1, + -1, 291, -1, 515, 294, 221, 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 236, -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, - 3039, -1, -1, -1, -1, -1, 3045, -1, -1, 454, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 3058, - -1, -1, 495, -1, -1, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, -1, -1, -1, -1, 512, + -1, -1, 176, -1, -1, -1, -1, -1, 274, -1, + -1, 277, -1, -1, -1, -1, -1, 2992, 2993, 193, + -1, -1, -1, -1, 198, 291, -1, -1, 294, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 3090, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 3017, 3018, -1, -1, -1, 221, 222, -1, + -1, 391, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 236, -1, -1, -1, -1, 3042, -1, -1, + -1, 3046, -1, 3048, 3049, 3050, -1, -1, 3053, -1, + -1, 3056, 3057, -1, -1, -1, -1, -1, -1, -1, + 3065, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 274, -1, -1, 277, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 291, -1, -1, + 294, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, 3106, -1, -1, -1, -1, -1, 3112, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 3116, -1, -1, + 3125, -1, -1, -1, -1, -1, -1, -1, 498, -1, + -1, 501, 502, 503, -1, 505, 506, 507, 508, 509, + 510, -1, -1, -1, -1, 515, -1, -1, -1, -1, + -1, -1, -1, 3158, -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, 3186, -1, -1, -1, -1, -1, 391, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 3150, 3151, 3152, -1, -1, -1, -1, -1, -1, + -1, -1, 498, -1, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, 3220, 3221, 3222, -1, 515, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 3170, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 3182, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 3240, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 3252, -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, 3226, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3243, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 3253, -1, -1, -1, -1, -1, + -1, -1, 3297, -1, 498, -1, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -1, -1, -1, + 3315, 515, -1, -1, -1, -1, -1, -1, -1, -1, + 3325, -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, 3278, + -1, -1, -1, -1, -1, 3350, -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, 3303, -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, - -1, -1, 39, -1, -1, 42, 43, 44, 3347, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, 3402, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 117, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, 159, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, -1, - -1, -1, -1, -1, -1, 502, 503, 504, -1, -1, - -1, -1, 509, -1, 511, -1, -1, -1, -1, 516, - 517, 518, 519, -1, 521, 3, 4, 5, 6, 7, - 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, + -1, 3376, -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, -1, -1, 39, + -1, -1, 42, 43, 44, 3420, 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, 72, 73, -1, 75, 76, 77, 78, 79, + -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, + 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, + 3475, 101, 102, 103, 104, 105, 106, 107, 108, 109, + 110, 111, 112, 113, 114, 115, 116, 117, 118, -1, + 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, + 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, + 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, + 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, + 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, + 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, + -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, + 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, + 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, + 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, + 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, + 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, + -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, + 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, + 310, 311, 312, 313, -1, -1, -1, -1, 318, 319, + 320, 321, 322, 323, 324, 325, 326, 327, -1, 329, + 330, 331, 332, 333, 334, -1, 336, 337, 338, 339, + 340, 341, 342, 343, -1, 345, 346, 347, 348, 349, + 350, 351, 352, 353, 354, 355, 356, 357, -1, 359, + 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, + 370, 371, 372, 373, 374, 375, 376, 377, 378, -1, + -1, 381, 382, 383, 384, 385, 386, 387, 388, 389, + -1, 391, 392, 393, 394, 395, -1, 397, 398, 399, + 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, + 410, 411, -1, -1, 414, 415, -1, 417, 418, 419, + 420, 421, 422, 423, -1, 425, 426, 427, -1, -1, + 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, + 440, 441, 442, 443, -1, -1, 446, 447, 448, -1, + 450, 451, 452, 453, -1, 455, 456, 457, 458, 459, + 460, 461, -1, 463, 464, 465, 466, 467, 468, 469, + 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, + 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, + 490, 491, 492, 493, 494, 495, 496, 497, 498, -1, + -1, -1, -1, -1, -1, 505, 506, 507, -1, -1, + -1, -1, 512, -1, 514, -1, -1, -1, -1, 519, + 520, 521, 522, -1, 524, 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, @@ -12077,7 +11188,7 @@ static const yytype_int16 yycheck[] = 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, @@ -12096,154 +11207,309 @@ static const yytype_int16 yycheck[] = 278, 279, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, -1, -1, - -1, -1, -1, -1, 502, 503, 504, -1, -1, -1, - -1, 509, -1, 511, -1, -1, -1, -1, 516, 517, - 518, 519, -1, 521, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - 159, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, 170, 171, 172, 173, 174, 175, 176, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 313, 314, 315, -1, 317, 318, - 319, 320, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, 376, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, 446, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, 461, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, 473, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, -1, 497, -1, - -1, -1, -1, 502, 503, 504, -1, -1, -1, -1, - 509, -1, 511, 512, -1, -1, -1, 516, 517, 518, - 519, 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, - 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, - -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, 117, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, 170, 171, - 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, - 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, - -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 315, -1, 317, 318, 319, 320, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, 376, -1, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, 446, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, 461, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, -1, 497, -1, -1, -1, -1, - 502, 503, 504, -1, -1, -1, -1, 509, -1, 511, - -1, -1, -1, -1, 516, 517, 518, 519, 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, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 498, -1, -1, -1, -1, -1, -1, 505, 506, 507, + -1, -1, -1, -1, 512, -1, 514, -1, -1, -1, + -1, 519, 520, 521, 522, -1, 524, 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, 72, 73, -1, 75, + 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, + 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, + 126, 127, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, + 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, + 236, -1, 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, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, 379, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, + 416, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, 449, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, -1, 500, -1, -1, -1, -1, 505, + 506, 507, -1, -1, -1, -1, 512, -1, 514, 515, + -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, 75, + 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, + 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, + 126, 127, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, + 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, + 236, -1, 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, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, 379, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, + 416, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, 449, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, -1, 500, -1, -1, -1, -1, 505, + 506, 507, -1, -1, -1, -1, 512, -1, 514, -1, + -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, 75, + 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, + 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, + 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, + 236, -1, 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, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, 379, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + 416, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, 449, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, -1, 500, -1, -1, -1, -1, 505, + 506, 507, -1, -1, -1, -1, 512, -1, 514, -1, + -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, 75, + 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, + 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, 132, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, 170, -1, 172, 173, 174, 175, + 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, + 236, -1, 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, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, -1, 380, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, 462, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 498, -1, -1, -1, -1, -1, -1, 505, + 506, 507, -1, -1, -1, -1, 512, -1, 514, 515, + -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, 75, + 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, + 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + 126, 127, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, + 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, + 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, + 236, -1, 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, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, + 316, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, 379, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, 390, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 412, -1, 414, 415, + 416, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, 449, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, -1, -1, 500, -1, -1, -1, 3, 4, + 5, 6, 7, 8, 9, 10, 512, -1, 514, -1, + -1, -1, -1, 519, 520, 521, 522, 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, + 55, 56, 57, 58, 59, 60, 61, 62, 63, 64, 65, 66, 67, 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, 132, -1, 134, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 165, -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, - 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 225, 226, 227, 228, 229, 230, -1, 232, 233, 234, 235, 236, -1, 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, @@ -12251,711 +11517,3055 @@ static const yytype_int16 yycheck[] = 275, 276, 277, 278, 279, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, 320, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, 360, 361, 362, 363, 364, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, 377, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - 415, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, 459, 460, 461, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, 429, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, 380, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, 462, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, 380, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + 515, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, 380, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + 515, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, 380, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, 171, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + 515, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, 416, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, 473, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, 416, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, 416, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, -1, -1, -1, -1, -1, -1, 502, 503, 504, - -1, -1, -1, -1, 509, -1, 511, 512, -1, -1, - -1, 516, 517, 518, 519, 3, 4, 5, 6, 7, - 8, 9, 10, -1, -1, -1, -1, -1, -1, -1, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, -1, -1, -1, -1, -1, -1, -1, + 505, 506, -1, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, -1, -1, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 278, 279, 280, 281, 282, 283, 284, + 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, 498, -1, -1, -1, -1, -1, -1, + 505, 506, 507, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, -1, -1, -1, -1, -1, -1, -1, + 505, 506, -1, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, -1, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, -1, -1, -1, -1, -1, -1, -1, + 505, 506, -1, -1, -1, -1, -1, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, -1, + 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, + 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, + 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, + 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, + 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, + 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, + 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, + 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, + 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, + 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, + 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, + -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, + 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, + 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, + 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, + 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, + 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, + -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, -1, + -1, -1, -1, 318, 319, 320, 321, 322, 323, 324, + 325, 326, 327, -1, 329, 330, 331, 332, 333, 334, + -1, 336, 337, 338, 339, 340, 341, 342, 343, -1, + 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, + 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, + 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, + 375, 376, 377, 378, -1, -1, 381, 382, 383, 384, + 385, 386, 387, 388, 389, -1, 391, 392, 393, 394, + 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, + 405, 406, 407, 408, 409, 410, 411, -1, -1, 414, + 415, -1, 417, 418, 419, 420, 421, 422, 423, -1, + 425, 426, 427, -1, -1, 430, 431, 432, 433, 434, + 435, 436, 437, 438, 439, 440, 441, 442, 443, -1, + -1, 446, 447, 448, -1, 450, 451, 452, 453, -1, + 455, 456, 457, 458, 459, 460, 461, -1, 463, 464, + 465, 466, 467, 468, 469, 470, 471, -1, -1, 474, + -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, + 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + 495, 496, 497, -1, -1, -1, -1, -1, -1, 3, + 4, 5, 6, 7, 8, 9, 10, 512, -1, 514, + -1, -1, -1, -1, 519, 520, 521, 522, 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, 72, 73, + -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, + 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, + 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, 115, 116, 117, 118, -1, 120, 121, 122, 123, + 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, + 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, + 144, -1, 146, -1, 148, 149, 150, 151, 152, 153, + 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, + 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, + 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, + 184, -1, 186, 187, 188, 189, 190, 191, 192, -1, + 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, + 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, -1, 218, -1, 220, -1, -1, 223, + 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, + 234, 235, -1, -1, 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, -1, 278, 279, 280, 281, 282, 283, + 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, + -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + -1, -1, -1, -1, 318, 319, 320, 321, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, + -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, -1, 359, 360, 361, 362, 363, + 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, + 374, 375, 376, 377, 378, -1, -1, 381, 382, 383, + 384, 385, 386, 387, 388, 389, -1, -1, 392, 393, + 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, -1, -1, + 414, 415, -1, 417, 418, 419, 420, 421, 422, 423, + -1, 425, 426, 427, -1, -1, 430, 431, 432, 433, + 434, 435, 436, 437, 438, 439, 440, 441, 442, 443, + -1, -1, 446, 447, 448, -1, 450, 451, 452, 453, + -1, 455, 456, 457, 458, 459, 460, 461, -1, 463, + 464, 465, 466, 467, 468, 469, 470, 471, -1, -1, + 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, -1, -1, -1, -1, -1, -1, + -1, 505, 506, 507, -1, 3, 4, 5, 512, -1, + 514, 9, -1, -1, -1, 519, 520, 521, 522, -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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, + 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, 159, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, + 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, + 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, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, 289, -1, 291, 292, 293, -1, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, 380, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, 435, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + -1, -1, -1, -1, -1, -1, -1, 505, 506, 507, + -1, 3, 4, 5, 6, 7, 514, 9, -1, -1, + -1, -1, -1, -1, 522, -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, + 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, + -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, + 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, + 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, + -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, + 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, + 172, 173, 174, 175, 176, 177, -1, 179, 180, 181, + 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, + 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, + 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, + 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, + -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, + 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, -1, -1, -1, -1, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, + 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, -1, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, -1, -1, 381, + 382, 383, 384, 385, 386, 387, 388, 389, -1, 391, + 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + -1, -1, 414, 415, -1, 417, 418, 419, 420, 421, + 422, 423, -1, 425, 426, 427, -1, -1, 430, 431, + 432, 433, 434, 435, 436, 437, 438, 439, 440, 441, + 442, 443, -1, -1, 446, 447, 448, -1, 450, 451, + 452, 453, -1, 455, 456, 457, 458, 459, 460, 461, + -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, -1, -1, -1, -1, + 3, 4, 5, 6, 7, -1, 9, -1, -1, -1, + 512, -1, 514, -1, -1, -1, -1, 519, 520, 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, + 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, + 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, + 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, + 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, 321, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, -1, -1, -1, -1, 3, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 512, + -1, 514, -1, -1, -1, -1, 519, 520, 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, 72, 73, + -1, 75, 76, 77, 78, 79, -1, -1, -1, 83, + 84, 85, 86, 87, 88, -1, 90, 91, 92, -1, + 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, + 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, + 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, 147, 148, 149, 150, 151, 152, 153, + 154, 155, 156, 157, 158, -1, 160, 161, 162, 163, + 164, 165, 166, 167, -1, 169, -1, -1, -1, 173, + 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, + 184, -1, 186, 187, 188, 189, 190, 191, 192, -1, + 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, + 204, 205, -1, 207, -1, 209, 210, 211, 212, 213, + 214, 215, 216, 217, 218, -1, 220, -1, -1, 223, + -1, 225, 226, 227, 228, 229, 230, -1, -1, 233, + -1, 235, -1, 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, + -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, + 284, 285, 286, 287, 288, -1, -1, 291, 292, 293, + -1, 295, 296, 297, 298, -1, 300, -1, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + -1, -1, -1, -1, 318, 319, 320, -1, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, + -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, + 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, + 374, 375, 376, 377, 378, -1, -1, 381, 382, 383, + 384, 385, 386, 387, 388, 389, -1, -1, 392, 393, + 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, -1, -1, + 414, 415, -1, 417, -1, 419, 420, 421, 422, 423, + -1, 425, 426, 427, -1, -1, 430, 431, 432, 433, + 434, -1, 436, 437, 438, 439, 440, 441, 442, 443, + 444, -1, 446, 447, 448, -1, 450, 451, 452, 453, + -1, 455, 456, 457, 458, 459, 460, 461, -1, 463, + -1, 465, 466, 467, 468, 469, 470, 471, -1, -1, + 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, 3, -1, 5, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 515, -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, -1, 75, 76, 77, 78, + 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, + -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, + 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, + -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, + 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, + 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, + -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, + 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, + 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, + 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, + 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, + 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, + -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, + 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, + -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, + -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, + -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, -1, -1, -1, -1, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, -1, + 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, + 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, + -1, -1, 381, 382, 383, 384, 385, 386, 387, 388, + 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, + 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, -1, -1, 414, 415, -1, 417, -1, + 419, 420, 421, 422, 423, -1, 425, 426, 427, -1, + -1, 430, 431, 432, 433, 434, -1, 436, 437, 438, + 439, 440, 441, 442, 443, -1, -1, 446, 447, 448, + -1, 450, 451, 452, 453, -1, 455, 456, 457, 458, + 459, 460, 461, -1, 463, -1, 465, 466, 467, 468, + 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, + 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, 3, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 514, 515, -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, 72, 73, + -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, + 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, + 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, + 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, + 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, + 124, 125, 126, 127, 128, 129, 130, 131, -1, -1, + 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, + 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, + 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, + 164, 165, -1, 167, -1, 169, 170, 171, 172, 173, + 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, + 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, + 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, + 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, + 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, + 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, + 234, 235, 236, -1, 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, -1, 280, 281, 282, 283, + 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, + -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 314, 315, 316, -1, 318, 319, 320, 321, 322, 323, + 324, 325, 326, 327, -1, 329, 330, 331, 332, 333, + 334, -1, 336, 337, 338, 339, 340, 341, 342, 343, + -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, + 354, 355, 356, 357, -1, 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, -1, 397, 398, 399, 400, 401, 402, 403, + 404, 405, 406, 407, 408, 409, 410, 411, 412, -1, + 414, 415, 416, 417, 418, 419, 420, 421, 422, 423, + -1, 425, 426, 427, -1, -1, 430, 431, 432, 433, + 434, -1, 436, 437, 438, 439, 440, 441, 442, 443, + -1, -1, 446, 447, 448, 449, 450, 451, 452, 453, + -1, 455, 456, 457, 458, 459, 460, 461, -1, 463, + 464, 465, 466, 467, 468, 469, 470, 471, -1, -1, + 474, -1, 476, 477, 478, 479, 480, 481, 482, 483, + 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, + 494, 495, 496, 497, -1, 3, 500, 5, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 514, -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, + -1, -1, -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, + -1, 59, 60, 61, 62, 63, 64, -1, 66, 67, 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, 126, 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, - 158, 159, 160, -1, 162, 163, 164, 165, -1, 167, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, -1, 232, 233, 234, 235, 236, -1, + 228, 229, 230, 231, -1, 233, 234, 235, 236, -1, 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, -1, 287, - 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 268, 269, 270, 271, 272, 273, -1, 275, 276, 277, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, 290, 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 308, 309, 310, 311, 312, 313, 314, 315, 316, -1, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, 379, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, 390, 391, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, 426, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, 412, -1, 414, 415, 416, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, 449, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, -1, -1, - -1, -1, -1, -1, 502, 503, 504, -1, -1, -1, - -1, 509, -1, 511, -1, -1, -1, -1, 516, 517, - 518, 519, 3, 4, 5, 6, 7, 8, 9, 10, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + -1, 3, 500, 5, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 514, -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, + 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, + -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, + 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, + 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, + -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, + 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, + 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, + 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, + 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, + 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, + 222, 223, 224, 225, 226, 227, 228, 229, 230, 231, + -1, 233, 234, 235, 236, -1, 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, -1, 280, 281, + 282, 283, 284, 285, -1, 287, 288, -1, 290, 291, + 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, -1, -1, -1, -1, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, + 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, -1, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, -1, -1, 381, + 382, 383, 384, 385, 386, 387, 388, 389, -1, 391, + 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + -1, -1, 414, 415, -1, 417, 418, 419, 420, 421, + 422, 423, -1, 425, 426, 427, -1, -1, 430, 431, + 432, 433, 434, -1, 436, 437, 438, 439, 440, 441, + 442, 443, -1, -1, 446, 447, 448, -1, 450, 451, + 452, 453, -1, 455, 456, 457, 458, 459, 460, 461, + -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 3, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 514, -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, 72, 73, -1, 75, 76, + 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, + 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, + 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, + -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, + -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, + 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, + 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, + 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, + 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, + 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, + 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, + -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, + 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, + -1, 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, -1, 280, 281, 282, 283, 284, 285, -1, + 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, + 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, -1, -1, -1, + -1, 318, 319, 320, 321, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, -1, 336, + 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, + 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, -1, 359, 360, 361, 362, 363, 364, 365, 366, + 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, + 377, 378, -1, -1, 381, 382, 383, 384, 385, 386, + 387, 388, 389, -1, 391, 392, 393, 394, 395, -1, + 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, -1, -1, 414, 415, -1, + 417, 418, 419, 420, 421, 422, 423, -1, 425, 426, + 427, -1, -1, 430, 431, 432, 433, 434, -1, 436, + 437, 438, 439, 440, 441, 442, 443, -1, -1, 446, + 447, 448, -1, 450, 451, 452, 453, -1, 455, 456, + 457, 458, 459, 460, 461, -1, 463, 464, 465, 466, + 467, 468, 469, 470, 471, -1, -1, 474, -1, -1, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, 3, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 514, -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, + 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, + -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, + 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, + 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, + -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, + 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, + 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, + 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, + 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, + 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, + 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, + -1, 233, 234, 235, 236, -1, 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, -1, 280, 281, + 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, + 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, -1, -1, -1, -1, 318, 319, 320, 321, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, + 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, -1, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, -1, -1, 381, + 382, 383, 384, 385, 386, 387, 388, 389, -1, 391, + 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + -1, -1, 414, 415, -1, 417, 418, 419, 420, 421, + 422, 423, -1, 425, 426, 427, -1, -1, 430, 431, + 432, 433, 434, -1, 436, 437, 438, 439, 440, 441, + 442, 443, -1, -1, 446, 447, 448, -1, 450, 451, + 452, 453, -1, 455, 456, 457, 458, 459, 460, 461, + -1, 463, 464, 465, 466, 467, 468, 469, 470, 471, + -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 3, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 514, -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, -1, 75, 76, + 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, + 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, + 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, + -1, 118, -1, 120, 121, 122, 123, 124, 125, 126, + 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, + 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, + 167, -1, 169, -1, 171, -1, 173, 174, 175, -1, + 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, + 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, + 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, + 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, + -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, + 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, + -1, 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, -1, 275, 276, + -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, + 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, + 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, + -1, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, -1, 336, + 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, + 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, + 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, + 377, 378, 379, -1, 381, 382, 383, 384, 385, 386, + 387, 388, 389, 390, -1, 392, 393, 394, 395, -1, + 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, 412, -1, 414, 415, 416, + 417, -1, 419, 420, 421, 422, 423, -1, 425, 426, + 427, -1, -1, 430, 431, 432, 433, 434, -1, 436, + 437, 438, 439, 440, 441, 442, 443, -1, -1, 446, + 447, 448, 449, 450, 451, 452, 453, -1, 455, 456, + 457, 458, 459, 460, 461, -1, 463, -1, 465, 466, + 467, 468, 469, 470, 471, -1, -1, 474, -1, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, 3, -1, 500, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 514, -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, + 72, 73, -1, 75, 76, 77, 78, 79, -1, -1, + -1, 83, 84, 85, 86, 87, 88, -1, 90, 91, + 92, -1, 94, 95, 96, 97, 98, 99, -1, -1, + 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, + 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, + 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, + -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, + 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, + 162, 163, 164, 165, -1, 167, -1, 169, -1, 171, + -1, 173, 174, 175, -1, 177, -1, 179, -1, 181, + 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, + 192, -1, 194, 195, 196, 197, -1, 199, 200, 201, + 202, 203, 204, 205, -1, 207, -1, 209, 210, 211, + 212, 213, 214, 215, 216, -1, 218, -1, 220, -1, + -1, 223, -1, 225, 226, 227, 228, 229, 230, -1, + -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, + 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, + 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, + 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, + 312, 313, 314, 315, 316, -1, 318, 319, 320, -1, + 322, 323, 324, 325, 326, 327, -1, 329, 330, 331, + 332, 333, 334, -1, 336, 337, 338, 339, 340, 341, + 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, + 352, 353, 354, 355, 356, 357, -1, 359, 360, -1, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, 379, -1, 381, + 382, 383, 384, 385, 386, 387, 388, 389, -1, -1, + 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, + 402, 403, 404, 405, 406, 407, 408, 409, 410, 411, + -1, -1, 414, 415, 416, 417, -1, 419, 420, 421, + 422, 423, -1, 425, 426, 427, -1, -1, 430, 431, + 432, 433, 434, -1, 436, 437, 438, 439, 440, 441, + 442, 443, -1, -1, 446, 447, 448, 449, 450, 451, + 452, 453, -1, 455, 456, 457, 458, 459, 460, 461, + -1, 463, -1, 465, 466, 467, 468, 469, 470, 471, + -1, -1, 474, -1, 476, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 3, -1, 500, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 514, -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, -1, 75, 76, + 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, + 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, + 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, + 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, + -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, + -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, + 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, + -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, + 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, + 167, -1, 169, -1, 171, -1, 173, 174, 175, -1, + 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, + 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, + 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, + 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, + -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, + 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, + -1, 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, -1, 275, 276, + -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, + 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, + 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, + 307, 308, 309, 310, 311, 312, 313, 314, 315, 316, + -1, 318, 319, 320, -1, 322, 323, 324, 325, 326, + 327, -1, 329, 330, 331, 332, 333, 334, -1, 336, + 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, + 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, + 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, + 367, 368, 369, 370, 371, 372, 373, 374, 375, 376, + 377, 378, 379, -1, 381, 382, 383, 384, 385, 386, + 387, 388, 389, -1, -1, 392, 393, 394, 395, -1, + 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, + 407, 408, 409, 410, 411, -1, -1, 414, 415, 416, + 417, -1, 419, 420, 421, 422, 423, -1, 425, 426, + 427, -1, -1, 430, 431, 432, 433, 434, -1, 436, + 437, 438, 439, 440, 441, 442, 443, -1, -1, 446, + 447, 448, 449, 450, 451, 452, 453, -1, 455, 456, + 457, 458, 459, 460, 461, -1, 463, -1, 465, 466, + 467, 468, 469, 470, 471, -1, -1, 474, -1, 476, + 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, + 487, 488, 489, 490, 491, 492, 493, 494, 495, 496, + 497, -1, 3, 500, 5, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 514, -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, + 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, -1, 75, 76, 77, 78, 79, -1, - 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, 117, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, - -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, - -1, -1, 233, 234, 235, 236, -1, 238, 239, 240, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, + 261, 262, 263, 264, 265, 266, 267, -1, 269, 270, + 271, 272, 273, -1, 275, 276, -1, 278, -1, 280, + 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 3, -1, 5, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 514, -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, -1, 75, + 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, + 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, + -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, + 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, + -1, -1, 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, -1, 275, + 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, -1, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, -1, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, -1, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, -1, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, -1, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 514, -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, -1, 75, 76, 77, 78, 79, -1, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, + 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, + 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, + 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, + 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, + 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 3, -1, 5, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 514, -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, -1, 75, + 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, + 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, + -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, + 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, + -1, -1, 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, -1, 275, + 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, -1, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, -1, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, -1, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, -1, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, -1, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 514, -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, -1, 75, 76, 77, 78, 79, -1, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, + 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, + 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, + 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, + 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, + 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 3, -1, 5, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 514, -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, -1, 75, + 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, + 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, + -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, + 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, + -1, -1, 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, -1, 275, + 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, -1, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, -1, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, -1, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, -1, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, -1, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 3, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 514, -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, -1, 75, 76, 77, 78, 79, -1, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, + 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, + 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, + 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, + 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, 320, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, 377, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, 415, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, 459, 460, - 461, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, -1, -1, -1, -1, -1, - -1, 502, 503, 504, -1, -1, -1, -1, 509, -1, - 511, -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, 117, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, 377, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, -1, -1, -1, -1, -1, -1, 502, 503, - 504, -1, -1, -1, -1, 509, -1, 511, 512, -1, - -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 117, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, 159, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 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, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, -1, - -1, -1, -1, -1, -1, 502, 503, 504, -1, -1, - -1, -1, 509, -1, 511, -1, -1, -1, -1, 516, - 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, 79, - -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, 117, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, - 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - 320, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, 377, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, -1, 414, 415, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, 461, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, -1, -1, -1, -1, - -1, -1, 502, 503, 504, -1, -1, -1, -1, 509, - -1, 511, 512, -1, -1, -1, 516, 517, 518, 519, - 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, 72, - 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, - 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, - -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - 113, 114, 115, 116, 117, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, - -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, - 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, - 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, - 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, - 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, - 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, - 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, - 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, - 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, - 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, 377, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, 415, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, 461, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, -1, -1, -1, -1, -1, -1, 502, - 503, 504, -1, -1, -1, -1, 509, -1, 511, -1, - -1, -1, -1, 516, 517, 518, 519, 3, 4, 5, - 6, 7, -1, 9, 10, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 22, 23, 24, 25, + 491, 492, 493, 494, 495, 496, 497, 3, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 514, -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, 72, 73, -1, 75, - 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, - 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, + -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, -1, 75, + 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, + 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, 170, 171, 172, 173, 174, 175, - 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, + 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, + -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, - 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, - 236, -1, 238, 239, 240, 241, 242, 243, 244, 245, + -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, + 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, + -1, -1, 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, + 266, 267, -1, 269, 270, 271, 272, 273, -1, 275, + 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, 313, 314, 315, - -1, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, + 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, 360, 361, 362, 363, 364, 365, + 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - 376, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, 413, 414, 415, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - 446, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, 461, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, 473, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, -1, - -1, 497, -1, -1, -1, 3, 4, 5, 6, 7, - 8, 9, 10, 509, -1, 511, -1, -1, -1, -1, - 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, - 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, - 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, - 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, - 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, - 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, - 158, 159, 160, -1, 162, 163, 164, 165, -1, 167, - -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, - -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, - 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, - -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, - 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, - 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, -1, 287, - 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, -1, 394, 395, 396, 397, - 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, 432, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, - 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 495, -1, -1, - -1, -1, -1, -1, 502, 503, 504, -1, -1, -1, - -1, 509, -1, 511, -1, -1, -1, -1, 516, 517, - 518, 519, 3, 4, 5, 6, 7, 8, 9, 10, + 376, 377, 378, -1, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, -1, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, -1, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, -1, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, -1, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 3, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 514, -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, -1, 75, 76, 77, 78, 79, -1, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, + 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, + 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, + 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, + 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, + 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, -1, 3, 4, + 5, -1, -1, 8, 9, -1, -1, -1, -1, -1, + 15, 16, -1, 514, 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, 77, 78, -1, 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, -1, 153, 154, + 155, 156, 157, -1, 159, 160, 161, 162, 163, 164, + 165, 166, 167, 168, 169, 170, 171, 172, 173, 174, + 175, 176, 177, 178, 179, 180, -1, -1, 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, -1, 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, -1, 281, 282, 283, 284, + 285, 286, 287, 288, 289, 290, 291, 292, 293, 294, + 295, 296, 297, -1, 299, 300, 301, -1, 303, 304, + 305, 306, 307, 308, 309, 310, 311, 312, 313, 314, + 315, 316, 317, 318, 319, -1, 321, 322, 323, -1, + 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, -1, 412, 413, 414, + 415, 416, 417, 418, 419, 420, 421, 422, 423, 424, + 425, 426, 427, 428, 429, 430, 431, -1, 433, -1, + 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, -1, 461, 462, 463, 464, + 465, 466, 467, 468, 469, 470, 471, 472, 473, 474, + 475, 476, 477, 478, 479, 480, 481, 482, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 494, + 495, 496, 497, -1, 3, -1, 501, 502, 503, 8, + 505, 506, 507, 508, 509, 510, 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, 72, 73, -1, 75, 76, 77, 78, + 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, + -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, + 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, + 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, + -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, + 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, + -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, + 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, + -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, + 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, + 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, + 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, + 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, + 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, + -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, + 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, + -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, + -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, + -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, + 309, 310, 311, 312, 313, -1, -1, -1, -1, 318, + 319, 320, -1, 322, 323, 324, 325, 326, 327, -1, + 329, 330, 331, 332, 333, 334, -1, 336, 337, 338, + 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, + 349, 350, 351, 352, 353, 354, 355, 356, 357, -1, + 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, + 369, 370, 371, 372, 373, 374, 375, 376, 377, 378, + -1, -1, 381, 382, 383, 384, 385, 386, 387, 388, + 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, + 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, + 409, 410, 411, -1, -1, 414, 415, -1, 417, -1, + 419, 420, 421, 422, 423, -1, 425, 426, 427, -1, + -1, 430, 431, 432, 433, 434, -1, 436, 437, 438, + 439, 440, 441, 442, 443, -1, -1, 446, 447, 448, + -1, 450, 451, 452, 453, -1, 455, 456, 457, 458, + 459, 460, 461, -1, 463, -1, 465, 466, 467, 468, + 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, + 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, + 489, 490, 491, 492, 493, 494, 495, 496, 497, -1, + -1, -1, 501, 502, 503, -1, 505, 506, 507, 508, + 509, 510, 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, 8, -1, -1, 11, -1, -1, 53, 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, 80, -1, -1, -1, -1, 46, + 8, -1, -1, 11, -1, -1, 53, 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, 80, -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, + -1, -1, 80, -1, -1, 36, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 46, 8, -1, -1, 11, + 176, -1, 53, 15, 16, 17, 18, 19, 20, 21, + -1, -1, -1, -1, -1, -1, -1, 193, -1, -1, + -1, -1, 198, -1, 36, -1, -1, -1, -1, 80, + -1, -1, -1, -1, 46, -1, -1, -1, -1, 176, + -1, 53, -1, -1, -1, 221, 222, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 193, -1, -1, -1, + 236, 198, -1, -1, -1, -1, -1, -1, 80, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 176, -1, + -1, -1, -1, -1, 221, 222, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 193, -1, -1, 274, 236, + 198, 277, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 291, -1, -1, 294, -1, + -1, -1, -1, 221, 222, 176, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 274, 236, -1, + 277, -1, 193, -1, -1, -1, -1, 198, -1, -1, + -1, -1, -1, -1, 291, -1, -1, 294, -1, -1, + -1, -1, -1, -1, 176, -1, -1, -1, -1, -1, + 221, 222, -1, -1, -1, -1, 274, -1, -1, 277, + -1, 193, -1, -1, -1, 236, 198, -1, -1, -1, + -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 221, + 222, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, -1, -1, 274, 236, -1, 277, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 291, -1, -1, 294, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 391, -1, -1, -1, -1, -1, + -1, -1, 274, -1, -1, 277, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 291, + -1, -1, 294, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 391, -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, 498, -1, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, -1, -1, -1, 515, + 391, -1, -1, -1, -1, -1, 8, -1, -1, 11, + -1, -1, -1, 15, 16, 17, 18, 19, 20, 21, + -1, 498, -1, -1, 501, 502, 503, -1, 505, 506, + 507, 508, 509, 510, 36, -1, -1, -1, 515, 391, + -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, + -1, 53, -1, -1, -1, -1, -1, -1, -1, -1, + 498, -1, -1, 501, 502, 503, -1, 505, 506, 507, + 508, 509, 510, -1, -1, -1, -1, 515, 80, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 8, -1, -1, 11, -1, -1, -1, 15, + 16, 17, 18, 19, 20, 21, -1, 498, -1, -1, + 501, 502, 503, -1, 505, 506, 507, 508, 509, 510, + 36, -1, -1, -1, 515, -1, -1, -1, -1, -1, + 46, 8, -1, -1, 11, -1, -1, 53, 15, 16, + 17, 18, 19, 20, 21, -1, 498, -1, -1, 501, + 502, 503, -1, 505, 506, 507, 508, 509, 510, 36, + -1, -1, -1, 515, 80, -1, -1, -1, -1, 46, + 8, -1, -1, 11, 176, -1, 53, 15, 16, 17, + 18, 19, 20, 21, -1, -1, -1, -1, -1, -1, + -1, 193, -1, -1, -1, -1, 198, -1, 36, -1, + -1, -1, -1, 80, -1, -1, -1, -1, 46, -1, + -1, -1, -1, -1, -1, 53, -1, -1, -1, 221, + 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 236, -1, -1, -1, -1, -1, + -1, -1, 80, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, + 176, -1, -1, 15, 16, 17, 18, 19, 20, 21, + -1, -1, 274, -1, -1, 277, -1, 193, -1, -1, + -1, -1, 198, -1, 36, -1, -1, -1, -1, 291, + -1, -1, 294, -1, 46, -1, -1, -1, -1, 176, + -1, 53, -1, -1, -1, 221, 222, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 193, -1, -1, -1, + 236, 198, -1, -1, -1, -1, -1, -1, 80, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 176, -1, + -1, -1, -1, -1, 221, 222, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 193, -1, -1, 274, 236, + 198, 277, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 291, -1, -1, 294, -1, + -1, -1, -1, 221, 222, -1, -1, -1, -1, 391, + -1, -1, -1, -1, -1, -1, -1, 274, 236, -1, + 277, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 291, -1, -1, 294, -1, -1, + -1, -1, -1, -1, 176, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 274, -1, -1, 277, + -1, 193, -1, -1, -1, -1, 198, -1, -1, -1, + -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 221, + 222, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, -1, -1, -1, 236, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 498, -1, -1, 501, + 502, 503, -1, 505, 506, 507, 508, 509, 510, -1, + -1, -1, -1, 515, 391, -1, -1, -1, -1, -1, + -1, -1, 274, -1, 8, 277, -1, 11, -1, -1, + -1, 15, 16, 17, 18, 19, 20, 21, -1, 291, + -1, -1, 294, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 36, 391, -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, -1, + -1, -1, 498, -1, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, 80, -1, -1, 515, + -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, + -1, -1, -1, 15, 16, 17, 18, 19, 20, 21, + -1, 498, -1, -1, 501, 502, 503, -1, 505, 506, + 507, 508, 509, 510, 36, -1, -1, -1, 515, 391, + -1, -1, -1, -1, 46, 8, -1, -1, 11, -1, + -1, 53, 15, 16, 17, 18, 19, 20, 21, -1, + 498, -1, -1, 501, 502, 503, -1, 505, 506, 507, + 508, 509, 510, 36, -1, -1, -1, 515, 80, -1, + -1, -1, -1, 46, -1, -1, -1, -1, -1, -1, + 53, -1, 176, -1, -1, 8, -1, -1, 11, -1, + -1, -1, 15, 16, 17, 18, 19, 20, 21, 193, + -1, -1, -1, -1, 198, -1, -1, 80, -1, -1, + -1, -1, -1, 36, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 46, -1, -1, -1, 221, 222, -1, + 53, -1, -1, -1, -1, -1, 498, -1, -1, 501, + 502, 503, 236, 505, 506, 507, 508, 509, 510, -1, + -1, -1, -1, 515, -1, -1, -1, 80, -1, -1, + -1, -1, -1, -1, 176, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 274, 193, -1, 277, -1, -1, 198, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 291, -1, -1, + 294, -1, -1, 176, -1, -1, -1, -1, -1, 221, + 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 193, -1, -1, -1, 236, 198, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 221, 222, + -1, -1, -1, 176, -1, -1, -1, -1, -1, -1, + -1, -1, 274, 236, -1, 277, -1, -1, -1, -1, + 193, -1, -1, -1, -1, 198, -1, -1, -1, 291, + -1, -1, 294, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 391, 221, 222, + -1, 274, -1, -1, 277, -1, -1, -1, -1, -1, + -1, -1, -1, 236, -1, -1, -1, -1, 291, -1, + -1, 294, -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, 274, -1, -1, 277, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 291, -1, + -1, 294, -1, -1, -1, -1, -1, -1, -1, 391, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 498, -1, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -1, 391, 513, -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, + -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, 391, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 498, -1, -1, 501, + 502, 503, -1, 505, 506, 507, 508, 509, 510, -1, + -1, 513, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 498, -1, -1, 501, 502, + 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, + 513, -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, 498, -1, -1, 501, 502, + 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, + 513, 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, -1, 75, 76, 77, 78, 79, -1, - 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + 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, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 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, - -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, - -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + 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, - -1, -1, 233, 234, 235, 236, -1, 238, 239, 240, + 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, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, + 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, -1, -1, -1, -1, 317, 318, 319, 320, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, 360, + 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, -1, -1, 378, 379, 380, + 371, 372, 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, 415, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, + 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, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - 461, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, + 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, 487, 488, 489, 490, - 491, 492, 493, 494, 495, -1, -1, -1, -1, -1, - -1, 502, 503, 504, -1, -1, -1, -1, 509, -1, - 511, -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, 117, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, 171, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, -1, -1, -1, -1, -1, -1, 502, 503, - 504, -1, -1, -1, -1, 509, -1, 511, -1, -1, - -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 117, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, 159, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, -1, - -1, -1, -1, -1, -1, 502, 503, 504, -1, -1, - -1, -1, 509, -1, 511, 512, -1, -1, -1, 516, - 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, 79, - -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, 117, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, - 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - 320, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, -1, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, 413, 414, 415, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, 461, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, -1, -1, -1, -1, - -1, -1, 502, 503, 504, -1, -1, -1, -1, 509, - -1, 511, -1, -1, -1, -1, 516, 517, 518, 519, - 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, 72, - 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, - 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, - -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, - 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, - 113, 114, 115, 116, 117, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, - -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, - 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, - 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, - 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, - 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, - 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, - 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, - 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, - 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, - 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, - 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, 415, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, 461, 462, - 463, 464, 465, 466, 467, 468, -1, 470, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 495, -1, -1, -1, -1, -1, -1, 502, - 503, 504, -1, -1, -1, -1, 509, -1, 511, -1, - -1, -1, -1, 516, 517, 518, 519, 3, 4, 5, - 6, 7, 8, 9, 10, -1, -1, -1, -1, -1, + 491, 492, 493, 494, 495, 496, 497, 3, -1, -1, + -1, -1, -1, -1, -1, -1, 507, -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, -1, 75, + 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, + 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, + 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, + 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, + 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, + 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, + 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, + -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, + -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, + 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, + 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, + -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, + 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, + 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, + -1, -1, 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, -1, 275, + 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, + -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, -1, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, + 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, + 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, + 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, + 376, 377, 378, -1, -1, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, -1, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, -1, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, -1, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, -1, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, + 496, 497, 3, -1, -1, -1, -1, -1, -1, -1, + -1, 507, -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, -1, 75, 76, 77, 78, 79, -1, + -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, + 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, + -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, + 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, + 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, + 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, + 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, + 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, + -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, + -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, + 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, + 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, + 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, + 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, + -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, + -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, + 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, + 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, + -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, + 311, 312, 313, -1, -1, -1, -1, 318, 319, 320, + -1, 322, 323, 324, 325, 326, 327, -1, 329, 330, + 331, 332, 333, 334, -1, 336, 337, 338, 339, 340, + 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, -1, 359, 360, + -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, + 371, 372, 373, 374, 375, 376, 377, 378, -1, -1, + 381, 382, 383, 384, 385, 386, 387, 388, 389, -1, + -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, + 401, 402, 403, 404, 405, 406, 407, 408, 409, 410, + 411, -1, -1, 414, 415, -1, 417, -1, 419, 420, + 421, 422, 423, -1, 425, 426, 427, -1, -1, 430, + 431, 432, 433, 434, -1, 436, 437, 438, 439, 440, + 441, 442, 443, -1, -1, 446, 447, 448, -1, 450, + 451, 452, 453, -1, 455, 456, 457, 458, 459, 460, + 461, -1, 463, -1, 465, 466, 467, 468, 469, 470, + 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, + 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 494, 495, 496, 497, 3, 4, 5, + -1, -1, -1, 9, -1, -1, 507, -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, + 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, 65, - 66, 67, 68, 69, 70, 71, 72, 73, -1, 75, + 56, 57, -1, 59, 60, 61, 62, 63, 64, -1, + -1, 67, 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, + 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, + -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, @@ -12969,923 +14579,607 @@ static const yytype_int16 yycheck[] = 236, -1, 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, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, + 266, 267, 268, 269, 270, 271, 272, 273, -1, 275, + 276, 277, 278, -1, 280, 281, 282, 283, 284, 285, + -1, 287, 288, 289, -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, + 306, 307, 308, 309, 310, 311, 312, 313, -1, -1, + -1, -1, 318, 319, 320, 321, 322, 323, 324, 325, + 326, 327, -1, 329, 330, 331, 332, 333, 334, -1, + 336, 337, 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, 360, 361, 362, 363, 364, 365, + 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, 415, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, 461, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, + 376, 377, 378, -1, 380, 381, 382, 383, 384, 385, + 386, 387, 388, 389, -1, 391, 392, 393, 394, 395, + -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, -1, -1, 414, 415, + -1, 417, 418, 419, 420, 421, 422, 423, -1, 425, + 426, 427, -1, -1, 430, 431, 432, 433, 434, 435, + 436, 437, 438, 439, 440, 441, 442, 443, -1, -1, + 446, 447, 448, -1, 450, 451, 452, 453, -1, 455, + 456, 457, 458, 459, 460, 461, -1, 463, 464, 465, + 466, 467, 468, 469, 470, 471, -1, -1, 474, -1, + -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - -1, -1, -1, -1, -1, -1, 502, 503, 504, -1, - -1, -1, -1, 509, -1, 511, -1, -1, -1, -1, - 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - 159, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, 170, -1, 172, 173, 174, 175, 176, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, 320, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, 413, 414, 415, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, 461, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, -1, -1, -1, - -1, -1, -1, 502, 503, 504, -1, -1, -1, -1, - 509, -1, 511, -1, -1, -1, -1, 516, 517, 518, - 519, 3, 4, 5, 6, 7, 8, 9, 10, -1, + 496, 497, -1, -1, 8, -1, -1, 11, -1, 505, + 506, 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, 8, -1, -1, 11, -1, -1, 53, + 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, 80, -1, -1, -1, + -1, 46, 8, -1, -1, 11, -1, -1, 53, 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, 80, -1, -1, -1, -1, + 46, -1, 126, -1, -1, -1, -1, 53, -1, -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, 80, -1, -1, -1, 36, -1, + -1, -1, 40, -1, -1, -1, -1, -1, 46, -1, + 8, -1, 176, 11, -1, 53, -1, 15, 16, 17, + 18, 19, 20, 21, -1, -1, -1, -1, -1, 193, + -1, -1, -1, -1, 198, -1, -1, -1, 36, -1, + -1, 166, 80, -1, -1, -1, 171, -1, 46, -1, + -1, 176, -1, -1, -1, 53, -1, 221, 222, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 193, -1, + -1, -1, 236, 198, -1, -1, -1, -1, -1, 165, + -1, -1, 80, -1, -1, -1, -1, -1, -1, -1, + 176, -1, -1, -1, -1, -1, 221, 222, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 193, -1, -1, + 274, 236, 198, 277, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 291, -1, -1, + 294, -1, -1, -1, -1, 221, 222, -1, 176, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 274, + 236, -1, 277, -1, -1, 193, -1, -1, -1, -1, + 198, -1, -1, -1, -1, -1, 291, -1, -1, 294, + -1, -1, -1, -1, -1, -1, -1, -1, 176, -1, + -1, -1, -1, 221, 222, -1, -1, -1, 274, -1, + -1, 277, -1, -1, -1, 193, -1, -1, 236, -1, + 198, -1, -1, -1, -1, 291, -1, -1, 294, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 221, 222, -1, -1, 391, -1, -1, + -1, -1, -1, -1, -1, -1, 274, 323, 236, 277, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 391, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 274, -1, -1, 277, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 454, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 317, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 498, -1, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -1, -1, -1, + -1, -1, -1, 391, -1, -1, -1, -1, 8, -1, + -1, 11, -1, -1, -1, 15, 16, 17, 18, 19, + 20, 21, -1, 498, -1, -1, 501, 502, 503, -1, + 505, 506, 507, 508, 509, 510, 36, -1, -1, -1, + 40, -1, -1, 391, -1, -1, 46, -1, -1, -1, + -1, -1, -1, 53, -1, -1, -1, -1, -1, -1, + -1, -1, 498, -1, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, -1, -1, -1, -1, + 80, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 8, -1, -1, 11, -1, + -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, + 498, -1, -1, 501, 502, 503, -1, 505, 506, 507, + 508, 509, 510, 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, + 498, -1, -1, 501, 502, 503, -1, 505, 506, 507, + 508, 509, 510, -1, -1, -1, -1, 80, -1, -1, + -1, -1, 8, -1, -1, 11, 176, -1, -1, 15, + 16, 17, 18, 19, 20, 21, -1, -1, -1, -1, + -1, -1, -1, 193, -1, -1, -1, -1, 198, -1, + 36, -1, -1, -1, 40, -1, -1, -1, -1, -1, + 46, -1, -1, -1, -1, -1, -1, 53, -1, -1, + -1, 221, 222, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 236, -1, -1, -1, + -1, -1, -1, -1, 80, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 171, 8, + -1, -1, 11, 176, -1, -1, 15, 16, 17, 18, + 19, 20, 21, -1, 274, -1, -1, 277, -1, -1, + 193, -1, -1, -1, -1, 198, -1, 36, -1, -1, + -1, 291, -1, -1, 294, -1, -1, 46, -1, -1, + -1, -1, -1, -1, 53, -1, -1, -1, 221, 222, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 236, -1, -1, -1, -1, -1, -1, + -1, 80, -1, -1, -1, -1, -1, -1, -1, -1, + 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 193, -1, -1, + -1, 274, 198, -1, 277, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 291, -1, + -1, 294, -1, -1, -1, 221, 222, -1, -1, -1, + -1, 391, -1, -1, 8, -1, -1, 11, -1, -1, + 236, 15, 16, 17, 18, 19, 20, 21, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 166, -1, -1, + -1, -1, 36, -1, -1, -1, -1, 176, -1, -1, + -1, -1, 46, -1, -1, -1, -1, -1, 274, 53, + -1, 277, -1, -1, 193, -1, -1, -1, -1, 198, + -1, -1, -1, -1, -1, 291, -1, -1, 294, -1, + -1, -1, -1, -1, -1, -1, 80, -1, -1, -1, + -1, -1, 221, 222, -1, -1, -1, -1, 391, -1, + -1, -1, -1, -1, -1, -1, -1, 236, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 498, -1, + -1, 501, 502, 503, -1, 505, 506, 507, 508, 509, + 510, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 274, -1, -1, 277, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 291, -1, -1, 294, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, -1, 176, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 193, + -1, -1, -1, -1, 198, 498, -1, -1, 501, 502, + 503, -1, 505, 506, 507, 508, 509, 510, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 221, 222, -1, + -1, -1, -1, -1, -1, -1, 8, -1, -1, 11, + -1, -1, 236, 15, 16, 17, 18, 19, 20, 21, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, 391, -1, 36, -1, -1, -1, 40, -1, + -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, + 274, 53, 498, 277, -1, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, -1, 291, -1, -1, + 294, -1, -1, -1, -1, -1, -1, -1, 80, -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, -1, -1, -1, -1, -1, -1, 46, -1, + -1, -1, -1, -1, -1, 53, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 498, + -1, -1, 501, 502, 503, -1, 505, 506, 507, 508, + 509, 510, 80, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 391, -1, 8, + -1, -1, 11, -1, 176, -1, 15, 16, 17, 18, + 19, 20, 21, -1, -1, -1, -1, -1, -1, -1, + -1, 193, -1, -1, -1, -1, 198, 36, -1, -1, + 424, -1, -1, -1, -1, -1, -1, 46, -1, -1, + -1, -1, -1, -1, 53, -1, -1, -1, -1, 221, + 222, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 236, -1, -1, -1, -1, -1, + -1, 80, -1, -1, -1, -1, -1, -1, 176, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 193, -1, -1, -1, -1, + 198, -1, 274, -1, 498, 277, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -1, -1, 291, + -1, -1, 294, 221, 222, -1, -1, -1, -1, -1, + -1, -1, 8, -1, -1, 11, -1, -1, 236, 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, 176, -1, -1, + 46, -1, -1, -1, -1, -1, 274, 53, -1, 277, + -1, -1, -1, -1, 193, -1, -1, -1, -1, 198, + -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, + -1, -1, -1, -1, 80, -1, -1, -1, -1, -1, + -1, -1, 221, 222, -1, -1, -1, -1, -1, 391, + -1, -1, -1, -1, -1, -1, -1, 236, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 8, -1, + -1, 11, -1, -1, -1, 15, 16, -1, -1, 19, + 20, 21, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 274, 36, -1, 277, -1, + -1, -1, -1, -1, -1, -1, 46, -1, -1, -1, + -1, -1, 291, 53, -1, 294, -1, -1, -1, -1, + -1, -1, -1, 391, 8, -1, -1, 11, -1, -1, + 176, 15, 16, -1, -1, 19, 20, 21, -1, -1, + 80, -1, -1, -1, -1, -1, -1, 193, -1, -1, + -1, -1, 198, -1, -1, -1, 498, -1, -1, 501, + 502, 503, 46, 505, 506, 507, 508, 509, 510, 53, + -1, 8, -1, -1, 11, 221, 222, -1, 15, 16, + -1, -1, 19, 20, 21, -1, -1, -1, -1, -1, + 236, -1, -1, -1, -1, -1, 80, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 46, + -1, 8, 391, -1, 11, -1, 53, -1, 15, 16, + -1, -1, 19, 20, 21, -1, -1, -1, 274, -1, + 498, 277, -1, 501, 502, 503, 176, 505, 506, 507, + 508, 509, 510, 80, -1, 291, -1, -1, 294, 46, + -1, -1, -1, 193, -1, -1, 53, -1, 198, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 221, 222, 80, -1, -1, -1, -1, -1, -1, + -1, -1, 176, -1, -1, -1, 236, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 193, + -1, -1, -1, -1, 198, -1, -1, -1, -1, 498, + -1, -1, 501, 502, 503, -1, 505, 506, 507, 508, + 509, 510, -1, -1, 274, -1, -1, 277, -1, 176, + -1, -1, -1, -1, -1, 391, -1, -1, -1, -1, + -1, 291, 236, -1, 294, -1, 193, -1, -1, -1, + -1, 198, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, 176, + -1, -1, -1, -1, 221, 222, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 193, -1, -1, 236, + -1, 198, -1, -1, -1, -1, -1, 291, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 221, 222, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 274, -1, 236, + 277, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 391, 498, -1, 291, 501, 502, 503, -1, 505, + 506, 507, 508, 509, 510, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 277, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 291, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 391, -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, 391, -1, -1, -1, 498, -1, + -1, 501, 502, 503, -1, 505, 506, 507, 508, 509, + 510, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 391, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, 498, -1, -1, 501, 502, 503, + -1, 505, 506, 507, 508, 509, 510, -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, 498, -1, -1, 501, 502, 503, -1, 505, 506, + 507, 508, 509, 510, -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, 498, -1, -1, 501, 502, 503, -1, 505, 506, + 507, 508, 509, 510, 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, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -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, - 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, - -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, 117, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, - 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, - 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, - -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, -1, -1, -1, -1, 317, 318, 319, 320, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, -1, -1, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, 413, 414, 415, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, -1, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, 461, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, -1, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 495, -1, -1, -1, -1, -1, -1, - 502, 503, 504, -1, -1, -1, -1, 509, -1, 511, - -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, - 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, - 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, - 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, - 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, 320, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - 415, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, 461, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, -1, -1, -1, -1, -1, -1, 502, 503, 504, - -1, -1, -1, -1, 509, -1, 511, -1, -1, -1, - -1, 516, 517, 518, 519, 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, + 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, - -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, - 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, - 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, - 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, - 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, + 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, -1, 120, 121, 122, 123, 124, 125, -1, 127, - 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 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, -1, 162, 163, 164, 165, -1, 167, - -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, - -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 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, - -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, - 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, + 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, -1, 287, - 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 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, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, + 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, -1, -1, + 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, -1, 394, 395, 396, 397, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, + 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, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 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, 487, - 488, 489, 490, 491, 492, 493, 494, 495, -1, -1, - -1, -1, -1, -1, 502, 503, 504, -1, -1, -1, - -1, 509, -1, 511, -1, -1, -1, -1, 516, 517, - 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, 117, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, - -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, - -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, 320, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, 415, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - 461, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 495, -1, -1, -1, -1, -1, - -1, 502, 503, 504, -1, -1, -1, -1, 509, -1, - 511, -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, 117, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 495, -1, -1, -1, -1, -1, -1, 502, 503, - 504, -1, -1, -1, -1, 509, -1, 511, -1, -1, - -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - 117, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, 159, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, 432, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 495, -1, - -1, -1, -1, -1, -1, 502, 503, 504, -1, -1, - -1, -1, 509, -1, 511, -1, -1, -1, -1, 516, - 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, 79, - -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, 117, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, - 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - 320, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, -1, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, -1, 414, 415, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, 461, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 495, -1, -1, -1, -1, - -1, -1, 502, 503, 504, -1, -1, -1, -1, 509, - -1, 511, -1, -1, -1, -1, 516, 517, 518, 519, - 3, 4, 5, 6, 7, 8, 9, 10, -1, -1, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 39, -1, -1, 42, - 43, 44, -1, 46, 47, 48, 49, 50, 51, 52, - 53, 54, 55, 56, 57, -1, 59, 60, 61, 62, + 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, -1, 75, 76, 77, 78, 79, -1, 81, -1, + 73, 74, 75, 76, 77, 78, 79, 80, 81, 82, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, - -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 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, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, - -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, - -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, - 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, - 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, - 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, - 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, - 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, - 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, - 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, - 233, 234, 235, 236, -1, 238, 239, 240, 241, 242, + 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, -1, 287, 288, -1, -1, 291, 292, - 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, + 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, - -1, -1, -1, -1, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, + 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, -1, -1, 378, 379, 380, 381, 382, + 373, 374, 375, 376, 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, 415, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, 461, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 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, 487, 488, 489, 490, 491, 492, - 493, 494, 495, -1, -1, -1, -1, -1, -1, 502, - 503, 504, -1, -1, -1, -1, 509, -1, 511, -1, - -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, 75, - 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, - 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, 170, -1, 172, 173, 174, 175, - 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, - 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, - 236, -1, 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, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, 415, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, 461, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 495, - -1, -1, -1, -1, -1, -1, 502, 503, 504, -1, - -1, -1, -1, 509, -1, 511, -1, -1, -1, -1, - 516, 517, 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 117, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - 159, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, 170, -1, 172, 173, 174, 175, 176, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, 320, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, 415, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, 461, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 495, -1, -1, -1, - -1, -1, -1, 502, 503, 504, -1, -1, -1, -1, - 509, -1, 511, -1, -1, -1, -1, 516, 517, 518, - 519, 3, 4, 5, 6, 7, 8, 9, 10, -1, + 493, 494, 495, 496, 497, 3, -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, - 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, - -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, - 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, - 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, - -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, -1, -1, -1, -1, 317, 318, 319, 320, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, -1, -1, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, -1, 414, 415, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, -1, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, 461, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, -1, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, -1, -1, -1, -1, -1, -1, -1, - 502, 503, -1, -1, -1, -1, -1, 509, -1, 511, - -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, - 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, 117, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, 159, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, - 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, 224, - 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, - 235, 236, -1, 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, -1, 278, 279, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, 320, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - 415, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, 432, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, 461, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 495, -1, -1, -1, -1, -1, -1, 502, 503, 504, - -1, -1, -1, -1, 509, -1, 511, -1, -1, -1, - -1, 516, 517, 518, 519, 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, + 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, - -1, 59, 60, 61, 62, 63, 64, 65, 66, 67, - 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, - 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, - 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, - 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, - 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, - 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 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, -1, 162, 163, 164, 165, -1, 167, - -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, - -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 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, - -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, - 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, + 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, -1, 275, 276, 277, - 278, 279, 280, 281, 282, 283, 284, 285, -1, 287, - 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 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, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, + 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, -1, -1, + 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, -1, 394, 395, 396, 397, + 388, 389, 390, 391, 392, 393, 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, + 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, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 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, 487, - 488, 489, 490, 491, 492, 493, 494, -1, -1, -1, - -1, -1, -1, -1, 502, 503, -1, -1, -1, -1, - -1, 509, -1, 511, -1, -1, -1, -1, 516, 517, - 518, 519, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, 159, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, - -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, 222, 223, 224, 225, 226, 227, 228, 229, 230, - -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, 320, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, 415, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, 432, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - 461, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, -1, -1, -1, -1, -1, -1, - -1, 502, 503, -1, -1, -1, -1, -1, 509, -1, - 511, -1, -1, -1, -1, 516, 517, 518, 519, 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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, -1, -1, -1, -1, -1, -1, 3, 4, 5, - 6, 7, 8, 9, 10, 509, -1, 511, -1, -1, - -1, -1, 516, 517, 518, 519, 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, 72, 73, -1, 75, - 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, - 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, 117, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, -1, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, 159, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, 170, -1, 172, 173, 174, 175, - -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, -1, -1, 223, 224, 225, - 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, - -1, -1, 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, -1, 278, 279, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, -1, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, 415, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, 432, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, 461, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, -1, - -1, -1, -1, -1, -1, -1, 502, 503, 504, -1, - 3, 4, 5, 509, -1, 511, 9, -1, -1, -1, - 516, 517, 518, 519, -1, -1, -1, -1, -1, 22, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, + 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, + 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, + 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, + 233, 234, 235, 236, -1, 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, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, 289, -1, 291, 292, + 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, 321, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, 380, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, 435, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, + 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, + 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, + 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, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, 380, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, + 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, 72, + 63, 64, -1, 66, 67, 68, 69, 70, 71, 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, - 153, 154, 155, 156, 157, 158, 159, 160, -1, 162, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, @@ -13898,671 +15192,475 @@ static const yytype_int16 yycheck[] = 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, -1, 280, 281, 282, - 283, 284, 285, -1, 287, 288, 289, -1, 291, 292, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, 321, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, 377, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, 415, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, 432, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, 461, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, -1, -1, -1, -1, -1, -1, -1, 502, - 503, 504, -1, 3, 4, 5, 6, 7, 511, 9, - -1, -1, -1, -1, -1, -1, 519, -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, 72, 73, -1, 75, 76, 77, 78, 79, - -1, 81, -1, 83, 84, 85, 86, 87, 88, 89, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, 159, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, - 180, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, 208, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, - 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, 299, - 300, 301, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - 320, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - 360, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, -1, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, 388, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, -1, 414, 415, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, 432, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, 461, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, -1, -1, -1, -1, 3, - 4, 5, 6, 7, -1, 9, -1, -1, -1, 509, - -1, 511, -1, -1, -1, -1, 516, 517, 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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, 159, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, 432, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, -1, -1, -1, -1, 3, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 509, -1, 511, -1, -1, - -1, -1, 516, 517, 22, 23, 24, 25, 26, 27, + 493, 494, 495, 496, 497, 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, -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, 72, 73, -1, 75, 76, 77, - 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, - 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, - 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, + 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, - 138, -1, 140, 141, 142, -1, 144, 145, 146, 147, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, - 158, -1, 160, 161, 162, 163, 164, 165, 166, 167, - -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, - 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, - -1, 209, 210, 211, 212, 213, 214, 215, 216, 217, - 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, - 228, 229, 230, -1, -1, 233, -1, 235, -1, 237, + 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, + 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, 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, -1, 275, 276, -1, - 278, -1, 280, 281, 282, 283, 284, 285, 286, 287, - 288, -1, -1, 291, 292, 293, -1, 295, 296, 297, - 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, -1, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, -1, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - -1, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 268, 269, 270, 271, 272, 273, -1, 275, 276, 277, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, -1, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, -1, 433, 434, 435, 436, 437, - 438, 439, 440, 441, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, -1, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 3, -1, 5, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 512, -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, -1, 75, - 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, - 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, - -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, - 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, - -1, -1, 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, -1, 275, - 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, -1, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, -1, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, -1, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, -1, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, -1, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, -1, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 3, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 511, 512, -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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, 171, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 280, 281, 282, 283, - 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, - 314, 315, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 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, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, 413, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, -1, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, 446, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, 473, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, -1, 3, 497, 5, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 511, -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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - 81, -1, 83, 84, 85, 86, 87, 88, 89, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, 170, - 171, 172, 173, 174, 175, 176, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, 208, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, - 231, -1, 233, 234, 235, 236, -1, 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, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, 290, - 291, 292, 293, -1, -1, 296, 297, 298, 299, 300, - 301, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, 313, 314, 315, -1, 317, 318, 319, 320, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, 360, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, 376, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, 388, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, 413, 414, 415, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, 446, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - 461, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, 473, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, -1, 3, 497, 5, -1, -1, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 78, 79, -1, -1, 82, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + 93, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, 168, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, 328, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, 445, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 511, -1, -1, -1, 22, 23, 24, 25, 26, 27, + -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, 72, 73, -1, 75, 76, 77, - 78, 79, -1, 81, -1, 83, 84, 85, 86, 87, - 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, - 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, + -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, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, 93, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, - -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, + 168, 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, - 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, - 208, 209, 210, 211, 212, 213, 214, 215, 216, -1, - 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, - 228, 229, 230, 231, -1, 233, 234, 235, 236, -1, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, 236, -1, 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, + -1, 269, 270, 271, 272, 273, -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, - 288, -1, 290, 291, 292, 293, -1, -1, 296, 297, - 298, 299, 300, 301, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, 320, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - 388, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + 328, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, -1, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, 445, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 3, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 511, -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, 72, 73, -1, 75, - 76, 77, 78, 79, -1, 81, -1, 83, 84, 85, - 86, 87, 88, 89, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, 170, -1, 172, 173, 174, 175, - 176, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, 208, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, - 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, - 236, -1, 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, -1, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, 299, 300, 301, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, 320, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, 360, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, 388, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, 415, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, -1, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, 461, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 3, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 511, -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, 72, 73, - -1, 75, 76, 77, 78, 79, -1, 81, -1, 83, - 84, 85, 86, 87, 88, 89, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, 170, -1, 172, 173, - 174, 175, 176, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, 208, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, - 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, - 234, 235, 236, -1, 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, -1, 280, 281, 282, 283, - 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, 299, 300, 301, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, 320, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, 360, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, 388, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, 415, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, -1, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, 461, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 3, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 511, -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, - 72, 73, -1, 75, 76, 77, 78, 79, -1, -1, - -1, 83, 84, 85, 86, 87, 88, -1, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, -1, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, -1, 171, - -1, 173, 174, 175, -1, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, -1, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, -1, - -1, 223, -1, 225, 226, 227, 228, 229, 230, -1, - -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, - 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, 313, 314, 315, -1, 317, 318, 319, -1, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, -1, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, 376, -1, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, -1, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, 413, 414, -1, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - -1, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, 446, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, -1, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, 473, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 3, -1, 497, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 511, - -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, -1, 75, 76, 77, 78, 79, - -1, -1, -1, 83, 84, 85, 86, 87, 88, -1, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - -1, 171, -1, 173, 174, 175, -1, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, -1, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, -1, -1, 223, -1, 225, 226, 227, 228, 229, - 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, - 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, - 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, 313, 314, 315, -1, 317, 318, 319, - -1, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - -1, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, 376, -1, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, -1, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, 413, 414, -1, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, -1, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, 446, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, -1, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, 473, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, -1, 3, 497, 5, -1, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 511, -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, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 3, -1, - 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 511, -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, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + -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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, 290, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, 290, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 511, -1, -1, 22, + -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, @@ -14573,7 +15671,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -14592,223 +15690,76 @@ static const yytype_int16 yycheck[] = 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, 416, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 3, -1, 5, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 511, -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, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 3, -1, 5, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 511, -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, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 3, -1, 5, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 511, -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, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 3, -1, + 493, 494, 495, 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 511, -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, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 511, -1, -1, 22, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, @@ -14819,7 +15770,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -14838,1282 +15789,668 @@ static const yytype_int16 yycheck[] = 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 3, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 511, -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, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, -1, 3, 4, 5, -1, -1, - 8, 9, -1, -1, -1, -1, -1, 15, 16, -1, - 511, 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, 77, - 78, -1, 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, -1, 153, 154, 155, 156, 157, - -1, 159, 160, 161, 162, 163, 164, 165, 166, 167, - 168, 169, 170, 171, 172, 173, 174, 175, 176, 177, - 178, 179, 180, -1, -1, 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, -1, 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, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, 283, 284, 285, 286, 287, - 288, 289, 290, 291, 292, 293, 294, 295, 296, 297, - -1, 299, 300, 301, -1, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, 313, 314, 315, 316, 317, - 318, -1, 320, 321, 322, -1, 324, 325, 326, 327, - 328, 329, 330, 331, 332, 333, 334, 335, 336, 337, - 338, 339, 340, 341, 342, 343, 344, 345, 346, 347, + -1, 269, 270, 271, 272, 273, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, - 358, 359, 360, 361, 362, 363, 364, 365, 366, 367, + -1, 359, 360, -1, 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, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - -1, 409, 410, 411, 412, 413, 414, 415, 416, 417, - 418, 419, 420, 421, 422, 423, 424, 425, 426, 427, - 428, -1, 430, -1, 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, -1, - 458, 459, 460, 461, 462, 463, 464, 465, 466, 467, - 468, 469, 470, 471, 472, 473, 474, 475, 476, 477, - 478, 479, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 491, 492, 493, 494, -1, 3, -1, - 498, 499, 500, 8, 502, 503, 504, 505, 506, 507, - 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - -1, -1, -1, 498, 499, 500, -1, 502, 503, 504, - 505, 506, 507, 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, 8, -1, -1, 11, -1, -1, 53, 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, 80, -1, -1, -1, -1, - 46, 8, -1, -1, 11, -1, -1, 53, 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, 80, -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, -1, -1, 80, -1, -1, 36, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 46, 8, -1, -1, - 11, 176, -1, 53, 15, 16, 17, 18, 19, 20, - 21, -1, -1, -1, -1, -1, -1, -1, 193, -1, - -1, -1, -1, 198, -1, 36, -1, -1, -1, -1, - 80, -1, -1, -1, -1, 46, -1, -1, -1, -1, - 176, -1, 53, -1, -1, -1, 221, 222, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 193, -1, -1, - -1, 236, 198, -1, -1, -1, -1, -1, -1, 80, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 176, - -1, -1, -1, -1, -1, 221, 222, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 193, -1, -1, 274, - 236, 198, 277, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 291, -1, -1, 294, - -1, -1, -1, -1, 221, 222, 176, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 274, 236, - -1, 277, -1, 193, -1, -1, -1, -1, 198, -1, - -1, -1, -1, -1, -1, 291, -1, -1, 294, -1, - -1, -1, -1, -1, -1, 176, -1, -1, -1, -1, - -1, 221, 222, -1, -1, -1, -1, 274, -1, -1, - 277, -1, 193, -1, -1, -1, 236, 198, -1, -1, - -1, -1, -1, -1, 291, -1, -1, 294, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 221, 222, -1, 388, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 274, 236, -1, 277, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 291, -1, -1, 294, -1, -1, -1, -1, -1, - -1, -1, 388, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 274, -1, -1, 277, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 291, -1, -1, 294, -1, -1, -1, -1, -1, -1, - -1, 388, -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, - 495, -1, -1, 498, 499, 500, -1, 502, 503, 504, - 505, 506, 507, -1, -1, -1, -1, 512, 388, -1, - -1, -1, -1, -1, 8, -1, -1, 11, -1, -1, - -1, 15, 16, 17, 18, 19, 20, 21, -1, 495, - -1, -1, 498, 499, 500, -1, 502, 503, 504, 505, - 506, 507, 36, -1, -1, -1, 512, 388, -1, -1, - -1, -1, 46, -1, -1, -1, -1, -1, -1, 53, - -1, -1, -1, -1, -1, -1, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, -1, -1, -1, 512, 80, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 8, -1, -1, 11, -1, -1, -1, 15, 16, 17, - 18, 19, 20, 21, -1, 495, -1, -1, 498, 499, - 500, -1, 502, 503, 504, 505, 506, 507, 36, -1, - -1, -1, 512, -1, -1, -1, -1, -1, 46, 8, - -1, -1, 11, -1, -1, 53, 15, 16, 17, 18, - 19, 20, 21, -1, 495, -1, -1, 498, 499, 500, - -1, 502, 503, 504, 505, 506, 507, 36, -1, -1, - -1, 512, 80, -1, -1, -1, -1, 46, 8, -1, - -1, 11, 176, -1, 53, 15, 16, 17, 18, 19, - 20, 21, -1, -1, -1, -1, -1, -1, -1, 193, - -1, -1, -1, -1, 198, -1, 36, -1, -1, -1, - -1, 80, -1, -1, -1, -1, 46, -1, -1, -1, - -1, -1, -1, 53, -1, -1, -1, 221, 222, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 236, -1, -1, -1, -1, -1, -1, -1, - 80, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 8, -1, -1, 11, 176, -1, - -1, 15, 16, 17, 18, 19, 20, 21, -1, -1, - 274, -1, -1, 277, -1, 193, -1, -1, -1, -1, - 198, -1, 36, -1, -1, -1, -1, 291, -1, -1, - 294, -1, 46, -1, -1, -1, -1, 176, -1, 53, - -1, -1, -1, 221, 222, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 193, -1, -1, -1, 236, 198, - -1, -1, -1, -1, -1, -1, 80, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 176, -1, -1, -1, - -1, -1, 221, 222, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 193, -1, -1, 274, 236, 198, 277, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, - -1, 221, 222, -1, 388, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 274, 236, -1, 277, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 291, -1, -1, 294, -1, -1, -1, -1, - -1, -1, 176, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 274, -1, -1, 277, -1, 193, - -1, -1, -1, -1, 198, -1, -1, -1, -1, -1, - -1, 291, -1, -1, 294, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 221, 222, -1, - 388, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 236, -1, -1, -1, -1, -1, -1, -1, - -1, 495, -1, -1, 498, 499, 500, -1, 502, 503, - 504, 505, 506, 507, -1, -1, -1, -1, 512, 388, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 8, - 274, -1, 11, 277, -1, -1, 15, 16, 17, 18, - 19, 20, 21, -1, -1, -1, -1, 291, -1, -1, - 294, -1, -1, -1, -1, -1, -1, 36, 388, -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, -1, -1, -1, 495, -1, -1, - 498, 499, 500, -1, 502, 503, 504, 505, 506, 507, - -1, 80, -1, -1, 512, -1, -1, -1, -1, -1, - -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, 495, -1, -1, 498, - 499, 500, -1, 502, 503, 504, 505, 506, 507, 36, - -1, -1, -1, 512, 388, -1, -1, -1, -1, 46, - 8, -1, -1, 11, -1, -1, 53, 15, 16, 17, - 18, 19, 20, 21, -1, 495, -1, -1, 498, 499, - 500, -1, 502, 503, 504, 505, 506, 507, 36, -1, - -1, -1, 512, 80, -1, -1, -1, -1, 46, -1, - -1, -1, -1, -1, -1, 53, -1, 176, -1, 8, - -1, -1, 11, -1, -1, -1, 15, 16, 17, 18, - 19, 20, 21, -1, 193, -1, -1, -1, -1, 198, - -1, -1, 80, -1, -1, -1, -1, 36, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 46, -1, -1, - -1, -1, 221, 222, 53, -1, -1, -1, -1, -1, - -1, 495, -1, -1, 498, 499, 500, 236, 502, 503, - 504, 505, 506, 507, -1, -1, -1, -1, 512, -1, - -1, 80, -1, -1, -1, -1, -1, -1, -1, 176, - -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, - 17, 18, 19, 20, 21, 274, 193, -1, 277, -1, - -1, 198, -1, -1, -1, -1, -1, -1, -1, 36, - -1, -1, 291, -1, -1, 294, -1, -1, 176, 46, - -1, -1, -1, -1, 221, 222, 53, -1, -1, -1, - -1, -1, -1, -1, -1, 193, -1, -1, -1, 236, - 198, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 80, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 221, 222, -1, -1, 176, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 274, 236, -1, - 277, -1, -1, -1, 193, -1, -1, -1, -1, 198, - -1, -1, -1, -1, 291, -1, -1, 294, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 388, - -1, -1, 221, 222, -1, -1, 274, -1, -1, 277, - -1, -1, -1, -1, -1, -1, -1, 236, -1, -1, - -1, -1, -1, 291, -1, -1, 294, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 176, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 274, 193, -1, 277, -1, - -1, 198, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 291, -1, -1, 294, -1, -1, -1, -1, - -1, 388, -1, -1, 221, 222, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 236, - -1, -1, -1, -1, -1, -1, 495, -1, -1, 498, - 499, 500, -1, 502, 503, 504, 505, 506, 507, -1, - 388, -1, -1, 512, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 274, -1, -1, - 277, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 291, -1, -1, 294, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 388, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, -1, 510, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 495, -1, -1, - 498, 499, 500, -1, 502, 503, 504, 505, 506, 507, - -1, -1, 510, -1, -1, -1, -1, -1, -1, -1, - -1, 388, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 495, -1, -1, 498, - 499, 500, -1, 502, 503, 504, 505, 506, 507, -1, - -1, 510, -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, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 3, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, -1, 510, 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, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 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, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, 487, - 488, 489, 490, 491, 492, 493, 494, 3, -1, -1, - -1, -1, -1, -1, -1, -1, 504, -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, -1, 75, - 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, - 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, - -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, - 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, - -1, -1, 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, -1, 275, - 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, -1, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, -1, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, -1, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, -1, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, -1, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, -1, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 3, - -1, -1, -1, -1, -1, -1, -1, -1, 504, -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, - -1, 75, 76, 77, 78, 79, -1, -1, -1, 83, - 84, 85, 86, 87, 88, -1, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, -1, -1, -1, 173, - 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, -1, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, -1, -1, 223, - -1, 225, 226, 227, 228, 229, 230, -1, -1, 233, - -1, 235, -1, -1, 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, - -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, - 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, -1, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, -1, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, -1, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, -1, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, -1, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, -1, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 3, 4, 5, -1, -1, -1, 9, -1, -1, - 504, -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, - 72, 73, -1, 75, 76, 77, 78, 79, -1, 81, - -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, 159, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, - 172, 173, 174, 175, 176, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, 208, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, - 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, - -1, 233, 234, 235, 236, -1, 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, -1, 280, 281, - 282, 283, 284, 285, -1, 287, 288, 289, -1, 291, - 292, 293, -1, -1, 296, 297, 298, 299, 300, 301, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, -1, -1, -1, -1, 317, 318, 319, 320, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, -1, 377, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, -1, 414, 415, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - 432, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, -1, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, 461, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, -1, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, -1, -1, 8, -1, -1, 11, -1, - 502, 503, 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, 8, -1, -1, 11, -1, -1, - 53, 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, 80, -1, -1, - -1, -1, 46, 8, -1, -1, 11, -1, -1, 53, - 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, 80, -1, -1, -1, - -1, 46, -1, 126, -1, -1, -1, -1, 53, -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, -1, 80, -1, -1, -1, -1, - 36, -1, -1, -1, 40, -1, -1, -1, -1, -1, - 46, -1, -1, 176, -1, -1, -1, 53, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 193, -1, -1, -1, -1, 198, -1, -1, -1, -1, - -1, -1, 166, -1, 80, -1, -1, 171, -1, -1, - -1, -1, 176, -1, -1, -1, -1, -1, 221, 222, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 193, - -1, -1, -1, 236, 198, -1, -1, -1, -1, -1, - 165, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 176, -1, -1, -1, -1, -1, 221, 222, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 193, -1, - -1, 274, 236, 198, 277, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 291, -1, - -1, 294, -1, -1, -1, -1, 221, 222, -1, -1, - 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 274, 236, -1, 277, -1, -1, -1, 193, -1, -1, - -1, -1, 198, -1, -1, -1, -1, 291, -1, -1, - 294, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 221, 222, -1, -1, 274, - -1, -1, 277, -1, -1, -1, -1, -1, -1, -1, - 236, -1, -1, -1, -1, -1, 291, -1, -1, 294, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 8, -1, -1, 11, 388, -1, -1, 15, 16, - 17, 18, 19, 20, 21, -1, -1, 322, 274, -1, - -1, 277, -1, -1, -1, -1, -1, -1, -1, 36, - -1, -1, -1, -1, -1, 291, -1, -1, 294, 46, - -1, -1, -1, -1, 388, -1, 53, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 451, 8, - -1, -1, 11, 80, -1, -1, 15, 16, 17, 18, - 19, 20, 21, 388, -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, 495, -1, 53, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, -1, -1, -1, -1, -1, - -1, -1, 388, -1, -1, -1, 8, -1, -1, 11, - -1, 80, -1, 15, 16, 17, 18, 19, 20, 21, - -1, 495, -1, -1, 498, 499, 500, -1, 502, 503, - 504, 505, 506, 507, 36, -1, -1, -1, -1, 176, - -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, - -1, 53, -1, -1, -1, -1, 193, -1, -1, -1, - 495, 198, -1, 498, 499, 500, -1, 502, 503, 504, - 505, 506, 507, -1, -1, -1, -1, -1, 80, -1, - -1, -1, -1, -1, 221, 222, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 236, - -1, -1, -1, -1, -1, -1, -1, 176, -1, 495, - -1, -1, 498, 499, 500, -1, 502, 503, 504, 505, - 506, 507, -1, -1, 193, -1, -1, -1, -1, 198, - -1, -1, -1, -1, -1, -1, -1, 274, -1, -1, - 277, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 221, 222, 291, -1, -1, 294, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 236, -1, 171, - -1, -1, -1, -1, 176, -1, -1, -1, -1, 316, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 193, -1, -1, -1, -1, 198, -1, -1, -1, - -1, -1, -1, -1, 8, 274, -1, 11, 277, -1, - -1, 15, 16, 17, 18, 19, 20, 21, -1, 221, - 222, -1, 291, -1, -1, 294, -1, -1, -1, -1, - -1, -1, 36, -1, 236, -1, 40, -1, -1, -1, - -1, -1, 46, -1, -1, -1, -1, -1, -1, 53, - -1, 388, -1, -1, -1, 8, -1, -1, 11, -1, - -1, -1, 15, 16, 17, 18, 19, 20, 21, -1, - -1, -1, 274, -1, -1, 277, 80, -1, -1, -1, - -1, -1, -1, 36, -1, -1, -1, -1, -1, 291, - -1, -1, 294, 46, -1, -1, -1, -1, -1, -1, - 53, -1, -1, -1, 8, -1, -1, 11, -1, -1, - -1, 15, 16, 17, 18, 19, 20, 21, -1, 388, - -1, -1, -1, -1, -1, -1, -1, 80, -1, -1, - -1, -1, 36, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 46, 8, -1, -1, 11, -1, -1, 53, - 15, 16, 17, 18, 19, 20, 21, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, 36, 176, -1, -1, 40, 80, -1, -1, -1, - -1, 46, -1, -1, -1, -1, 388, -1, 53, 193, - -1, -1, -1, -1, 198, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 166, -1, 80, -1, 221, 222, -1, - -1, -1, -1, 176, -1, -1, 495, -1, -1, 498, - 499, 500, 236, 502, 503, 504, 505, 506, 507, -1, - 193, -1, -1, 8, -1, 198, 11, -1, -1, -1, - 15, 16, 17, 18, 19, 20, 21, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 221, 222, - 274, 36, 176, 277, -1, -1, -1, -1, -1, -1, - -1, 46, -1, 236, -1, -1, -1, 291, 53, 193, - 294, -1, -1, 495, 198, -1, 498, 499, 500, -1, - 502, 503, 504, 505, 506, 507, -1, -1, -1, -1, - -1, 176, -1, -1, -1, 80, -1, 221, 222, -1, - -1, 274, -1, -1, 277, -1, -1, -1, 193, -1, - -1, -1, 236, 198, -1, -1, -1, -1, 291, -1, - -1, 294, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 221, 222, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 274, 236, -1, 277, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, 388, -1, -1, 291, -1, -1, - 294, -1, 8, -1, -1, 11, -1, -1, -1, 15, - 16, 17, 18, 19, 20, 21, -1, -1, -1, 274, - -1, 176, 277, -1, -1, -1, -1, -1, -1, -1, - 36, -1, -1, -1, -1, -1, 291, -1, 193, 294, - 46, -1, -1, 198, -1, 388, -1, 53, -1, -1, - -1, 8, -1, -1, 11, -1, -1, -1, 15, 16, - -1, -1, 19, 20, 21, -1, 221, 222, -1, -1, - -1, -1, -1, -1, 80, -1, -1, -1, -1, 36, - -1, 236, -1, -1, -1, -1, -1, -1, -1, 46, - -1, -1, -1, -1, 388, -1, 53, -1, -1, -1, - -1, 495, -1, -1, 498, 499, 500, -1, 502, 503, - 504, 505, 506, 507, -1, -1, -1, -1, -1, 274, - -1, -1, 277, 80, -1, -1, -1, 421, -1, -1, - -1, -1, -1, 388, -1, -1, 291, -1, -1, 294, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 495, -1, -1, 498, 499, 500, -1, 502, - 503, 504, 505, 506, 507, -1, -1, -1, -1, -1, - 176, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 8, 193, -1, 11, - -1, -1, 198, 15, 16, -1, -1, 19, 20, 21, - -1, 495, -1, -1, 498, 499, 500, -1, 502, 503, - 504, 505, 506, 507, 36, 221, 222, -1, -1, 176, - -1, -1, -1, -1, 46, -1, -1, -1, -1, -1, - 236, 53, -1, 388, -1, -1, 193, -1, -1, -1, - 495, 198, -1, 498, 499, 500, -1, 502, 503, 504, - 505, 506, 507, -1, -1, -1, -1, -1, 80, -1, - -1, -1, -1, -1, 221, 222, -1, -1, 274, -1, - -1, 277, -1, -1, -1, -1, -1, -1, -1, 236, - -1, -1, -1, -1, -1, 291, -1, -1, 294, -1, - 8, -1, -1, 11, -1, -1, -1, 15, 16, -1, - -1, 19, 20, 21, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 274, -1, -1, - 277, -1, -1, -1, -1, -1, -1, -1, 46, -1, - -1, -1, -1, -1, 291, 53, -1, 294, -1, -1, - 495, -1, -1, 498, 499, 500, -1, 502, 503, 504, - 505, 506, 507, -1, 176, -1, -1, -1, -1, -1, - -1, -1, 80, -1, -1, -1, -1, 8, -1, -1, - 11, 193, -1, -1, 15, 16, 198, -1, 19, 20, - 21, -1, 388, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 221, - 222, -1, -1, -1, -1, 46, -1, -1, -1, -1, - -1, -1, 53, -1, 236, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, 388, -1, -1, -1, -1, -1, -1, -1, 80, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 274, -1, -1, 277, -1, -1, 176, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 291, - -1, -1, -1, -1, -1, 193, -1, -1, -1, -1, - 198, -1, -1, -1, -1, -1, -1, -1, -1, 495, - -1, -1, 498, 499, 500, -1, 502, 503, 504, 505, - 506, 507, -1, 221, 222, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 236, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 176, -1, -1, 495, -1, - -1, 498, 499, 500, -1, 502, 503, 504, 505, 506, - 507, -1, 193, -1, -1, -1, 274, 198, -1, 277, - -1, -1, -1, -1, -1, -1, 388, -1, -1, -1, - -1, -1, -1, 291, -1, -1, -1, -1, -1, -1, - 221, 222, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 236, -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, 277, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 291, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 388, -1, -1, 495, -1, -1, 498, 499, 500, -1, - 502, 503, 504, 505, 506, 507, -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, 388, -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, 495, -1, -1, - 498, 499, 500, -1, 502, 503, 504, 505, 506, 507, - -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, 3, -1, - 5, -1, -1, -1, 495, -1, -1, 498, 499, 500, - -1, 502, 503, 504, 505, 506, 507, 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, 487, 488, 489, 490, 491, 492, 493, 494, - 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 269, 270, 271, 272, 273, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 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, + 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, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 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, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, + 263, 264, 265, 266, 267, -1, 269, 270, 271, 272, + 273, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 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, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 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, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, 81, -1, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - 159, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, 170, -1, 172, 173, 174, 175, 176, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, -1, -1, 233, 234, 235, 236, -1, 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, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - 289, -1, 291, 292, 293, -1, -1, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, 320, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, 360, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, 377, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, 415, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, 432, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, 461, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, 299, 300, 301, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, 320, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 3, -1, + 493, 494, 495, 496, 497, 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, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, 81, -1, 83, 84, - 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, - 175, 176, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, 208, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, - 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, - 235, 236, -1, 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, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, 299, 300, 301, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, 320, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - 415, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, 461, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 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, + 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, 72, - 73, -1, 75, 76, 77, 78, 79, -1, 81, -1, - 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, - -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, - 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, - 173, 174, 175, 176, 177, -1, 179, -1, 181, 182, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, - 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, - 203, 204, 205, -1, 207, 208, 209, 210, 211, 212, - 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, - 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, - 233, 234, 235, 236, -1, 238, 239, 240, 241, 242, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 280, 281, 282, + 263, 264, 265, 266, 267, -1, 269, 270, 271, 272, + 273, -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, - 293, -1, -1, 296, 297, 298, 299, 300, 301, 302, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, 360, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, 388, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, 415, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, 461, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 78, 79, -1, - -1, 82, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, 93, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, 168, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, 327, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, 442, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 3, -1, -1, -1, -1, -1, + 493, 494, 495, 496, 497, 3, -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, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, 93, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, 168, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, 236, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, 327, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, 442, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 3, -1, 5, -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, 72, 73, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 72, + 73, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -1, 5, -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, - -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, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 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, @@ -16126,7 +16463,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -16142,224 +16479,77 @@ static const yytype_int16 yycheck[] = 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, 263, 264, 265, 266, 267, -1, 269, 270, 271, 272, 273, -1, 275, 276, -1, 278, -1, 280, 281, 282, - 283, 284, 285, -1, 287, 288, -1, 290, 291, 292, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, 290, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 3, -1, 5, -1, -1, -1, + 493, 494, 495, 496, 497, 3, -1, 5, -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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, 413, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 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, @@ -16372,7 +16562,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -16385,228 +16575,129 @@ static const yytype_int16 yycheck[] = 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, - 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, - 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, - 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, - 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 3, -1, 5, -1, -1, -1, -1, -1, -1, -1, + 253, 254, 255, 256, 257, 258, 259, 260, 261, 262, + 263, 264, 265, 266, 267, -1, 269, 270, 271, 272, + 273, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, 72, 73, -1, 75, 76, 77, + 78, -1, -1, 81, -1, 83, 84, 85, 86, 87, + 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, -1, 153, 154, 155, 156, 157, + -1, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, + -1, 179, -1, -1, -1, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + 208, -1, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, + 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, + 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, -1, 275, 276, 277, + 278, -1, -1, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + -1, 299, 300, 301, -1, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, -1, 321, 322, 323, -1, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, 380, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, -1, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, -1, 433, -1, -1, 436, 437, + 438, 439, 440, 441, 442, 443, 444, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, -1, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 3, -1, 5, -1, -1, + -1, -1, -1, -1, -1, -1, 494, 495, 496, 497, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, @@ -16618,7 +16709,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -16637,225 +16728,177 @@ static const yytype_int16 yycheck[] = 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 3, -1, 5, -1, -1, -1, + 493, 494, 495, 496, 497, 3, -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, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 3, -1, 5, -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, + -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, 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, + 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, @@ -16864,7 +16907,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -16883,225 +16926,78 @@ static const yytype_int16 yycheck[] = 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 3, -1, 5, -1, + 493, 494, 495, 496, 497, 3, -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, 72, 73, -1, 75, 76, - 77, 78, 79, -1, -1, -1, 83, 84, 85, 86, - 87, 88, -1, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, -1, -1, -1, 173, 174, 175, -1, - 177, -1, 179, -1, 181, 182, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, -1, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, -1, 209, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, -1, -1, 223, -1, 225, 226, - 227, 228, 229, 230, -1, -1, 233, -1, 235, -1, - -1, 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, -1, 275, 276, - -1, 278, -1, 280, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, 319, -1, 321, 322, 323, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, -1, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - -1, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, -1, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, 408, 409, -1, 411, 412, -1, 414, -1, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, 429, 430, 431, -1, 433, 434, 435, 436, - 437, 438, 439, 440, -1, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - 457, 458, -1, 460, -1, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, - 487, 488, 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, - 75, 76, 77, 78, 79, -1, -1, -1, 83, 84, - 85, 86, 87, 88, -1, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, - 155, 156, 157, 158, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, -1, -1, -1, 173, 174, - 175, -1, 177, -1, 179, -1, 181, 182, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, -1, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, -1, 209, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, -1, -1, 223, -1, - 225, 226, 227, 228, 229, 230, -1, -1, 233, -1, - 235, -1, -1, 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, -1, - 275, 276, -1, 278, -1, 280, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, 319, -1, 321, 322, 323, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, -1, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, -1, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, -1, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, 408, 409, -1, 411, 412, -1, 414, - -1, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, 429, 430, 431, -1, 433, 434, - 435, 436, 437, 438, 439, 440, -1, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, 457, 458, -1, 460, -1, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 3, -1, 5, -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, -1, 75, 76, 77, + 78, 79, -1, -1, -1, 83, 84, 85, 86, 87, + 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, + 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, + 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, + 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, + 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, + 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, + 158, -1, 160, -1, 162, 163, 164, 165, -1, 167, + -1, 169, -1, -1, -1, 173, 174, 175, -1, 177, + -1, 179, -1, 181, 182, 183, 184, -1, 186, 187, + 188, 189, 190, 191, 192, -1, 194, 195, 196, 197, + -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, + -1, 209, 210, 211, 212, 213, 214, 215, 216, -1, + 218, -1, 220, -1, -1, 223, -1, 225, 226, 227, + 228, 229, 230, -1, -1, 233, -1, 235, -1, -1, + 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, -1, 275, 276, -1, + 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, + 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, + 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, + 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, + 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, 72, @@ -17110,7 +17006,7 @@ static const yytype_int16 yycheck[] = -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, - 123, 124, 125, -1, 127, 128, 129, 130, 131, -1, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, @@ -17129,220 +17025,25 @@ static const yytype_int16 yycheck[] = 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, - -1, -1, -1, -1, 317, 318, 319, -1, 321, 322, - 323, 324, 325, 326, -1, 328, 329, 330, 331, 332, - 333, -1, 335, 336, 337, 338, 339, 340, 341, 342, - -1, 344, 345, 346, 347, 348, 349, 350, 351, 352, - 353, 354, 355, 356, -1, 358, 359, -1, 361, 362, - 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, - 373, 374, 375, -1, -1, 378, 379, 380, 381, 382, - 383, 384, 385, 386, 387, -1, 389, 390, 391, 392, - -1, 394, 395, 396, 397, 398, 399, 400, 401, 402, - 403, 404, 405, 406, 407, 408, 409, -1, 411, 412, - -1, 414, -1, 416, 417, 418, 419, 420, -1, 422, - 423, 424, -1, -1, 427, 428, 429, 430, 431, -1, - 433, 434, 435, 436, 437, 438, 439, 440, -1, -1, - 443, 444, 445, -1, 447, 448, 449, 450, -1, 452, - 453, 454, 455, 456, 457, 458, -1, 460, -1, 462, - 463, 464, 465, 466, 467, 468, -1, -1, 471, -1, - -1, 474, 475, 476, 477, 478, 479, 480, 481, 482, - 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, - 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, 183, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, 309, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, 336, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, 365, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, - 79, -1, -1, -1, 83, 84, 85, 86, 87, 88, - -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, 152, 153, 154, 155, 156, 157, 158, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, -1, -1, -1, 173, 174, 175, -1, 177, -1, - 179, -1, 181, 182, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, -1, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, -1, - 209, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, -1, -1, 223, -1, 225, 226, 227, 228, - 229, 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, - -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, - -1, 300, -1, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - 319, -1, 321, 322, 323, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, -1, 361, 362, 363, 364, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, -1, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, -1, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, 406, 407, 408, - 409, -1, 411, 412, -1, 414, -1, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - 429, 430, 431, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, 457, 458, - -1, 460, -1, 462, 463, 464, 465, 466, 467, 468, - -1, -1, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 480, 481, 482, 483, 484, 485, 486, 487, 488, - 489, 490, 491, 492, 493, 494, 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, 72, 73, -1, 75, 76, - 77, 78, -1, -1, 81, -1, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, -1, 94, 95, 96, - 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, - 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, - -1, 118, -1, 120, 121, 122, 123, 124, 125, -1, - 127, 128, 129, 130, 131, -1, -1, 134, 135, 136, - 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, - -1, 148, 149, 150, 151, -1, 153, 154, 155, 156, - 157, -1, -1, 160, -1, 162, 163, 164, 165, -1, - 167, -1, 169, 170, -1, 172, 173, 174, 175, 176, - 177, -1, 179, -1, -1, -1, 183, 184, -1, 186, - 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, - 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, - 207, 208, -1, 210, 211, 212, 213, 214, 215, 216, - -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, - 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, - -1, 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, -1, 275, 276, - 277, 278, -1, -1, 281, 282, 283, 284, 285, -1, - 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, - 297, -1, 299, 300, 301, -1, 303, 304, 305, 306, - 307, 308, 309, 310, 311, 312, -1, -1, -1, -1, - 317, 318, -1, 320, 321, 322, -1, 324, 325, 326, - -1, 328, 329, 330, 331, 332, 333, -1, 335, 336, - 337, 338, 339, 340, 341, 342, -1, 344, 345, 346, - 347, 348, 349, 350, 351, 352, 353, 354, 355, 356, - -1, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, -1, - 377, 378, 379, 380, 381, 382, 383, 384, 385, 386, - 387, 388, 389, 390, 391, 392, -1, 394, 395, 396, - 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, - 407, -1, 409, -1, 411, 412, -1, 414, 415, 416, - 417, 418, 419, 420, -1, 422, 423, 424, -1, -1, - 427, 428, -1, 430, -1, -1, 433, 434, 435, 436, - 437, 438, 439, 440, 441, -1, 443, 444, 445, -1, - 447, 448, 449, 450, -1, 452, 453, 454, 455, 456, - -1, 458, -1, 460, 461, 462, 463, 464, 465, 466, - 467, 468, -1, -1, 471, -1, -1, 474, 475, 476, - 477, 478, 479, 3, -1, 5, -1, -1, -1, -1, - -1, -1, -1, -1, 491, 492, 493, 494, -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, -1, 75, 76, 77, 78, 79, - -1, -1, -1, 83, 84, 85, 86, 87, 88, -1, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - -1, -1, -1, 173, 174, 175, -1, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, -1, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, -1, -1, 223, -1, 225, 226, 227, 228, 229, - 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, - 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, - 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - -1, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - -1, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, -1, 378, 379, - 380, 381, 382, 383, 384, 385, 386, 387, -1, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, -1, 414, -1, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, -1, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, -1, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 3, -1, -1, -1, -1, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, -1, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, @@ -17354,7 +17055,7 @@ static const yytype_int16 yycheck[] = 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, @@ -17373,222 +17074,75 @@ static const yytype_int16 yycheck[] = 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, -1, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, -1, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - -1, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, -1, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, -1, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, -1, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 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, -1, 75, - 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, - 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, - -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, - 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, - -1, -1, 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, -1, 275, - 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, -1, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, -1, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, -1, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, -1, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, -1, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, -1, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 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, - -1, 75, 76, 77, 78, 79, -1, -1, -1, 83, - 84, 85, 86, 87, 88, -1, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, -1, -1, -1, 173, - 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, -1, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, -1, -1, 223, - -1, 225, 226, 227, 228, 229, 230, -1, -1, 233, - -1, 235, -1, -1, 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, - -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, - 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, -1, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, -1, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, -1, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, -1, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, -1, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, -1, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 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, -1, 75, 76, 77, 78, 79, -1, -1, - -1, 83, 84, 85, 86, 87, 88, -1, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, -1, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - 152, 153, 154, 155, 156, 157, 158, -1, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, -1, -1, - -1, 173, 174, 175, -1, 177, -1, 179, -1, 181, - 182, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, -1, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, -1, 209, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, -1, - -1, 223, -1, 225, 226, 227, 228, 229, 230, -1, - -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, - 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, - 302, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, -1, -1, -1, -1, 317, 318, 319, -1, 321, - 322, 323, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, -1, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, -1, -1, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, -1, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, 408, 409, -1, 411, - 412, -1, 414, -1, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, 429, 430, 431, - -1, 433, 434, 435, 436, 437, 438, 439, 440, -1, - -1, 443, 444, 445, -1, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, 457, 458, -1, 460, -1, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, -1, 474, 475, 476, 477, 478, 479, 480, 481, - 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, - 492, 493, 494, 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, 72, 73, -1, 75, 76, 77, 78, 79, - -1, -1, -1, 83, 84, 85, 86, 87, 88, -1, - 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, - -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, - 110, 111, 112, 113, 114, 115, 116, -1, 118, -1, - 120, 121, 122, 123, 124, 125, -1, 127, 128, 129, - 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, - 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, - 150, 151, 152, 153, 154, 155, 156, 157, 158, -1, - 160, -1, 162, 163, 164, 165, -1, 167, -1, 169, - -1, -1, -1, 173, 174, 175, -1, 177, -1, 179, - -1, 181, 182, 183, 184, -1, 186, 187, 188, 189, - 190, 191, 192, -1, 194, 195, 196, 197, -1, 199, - 200, 201, 202, 203, 204, 205, -1, 207, -1, 209, - 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, - 220, -1, -1, 223, -1, 225, 226, 227, 228, 229, - 230, -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, - 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, - -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, - 300, -1, 302, 303, 304, 305, 306, 307, 308, 309, - 310, 311, 312, -1, -1, -1, -1, 317, 318, 319, - -1, 321, 322, 323, 324, 325, 326, -1, 328, 329, - 330, 331, 332, 333, -1, 335, 336, 337, 338, 339, - 340, 341, 342, -1, 344, 345, 346, 347, 348, 349, - 350, 351, 352, 353, 354, 355, 356, -1, 358, 359, - -1, 361, 362, 363, 364, 365, 366, 367, 368, 369, - 370, 371, 372, 373, 374, 375, -1, -1, 378, 379, - 380, 381, 382, -1, 384, 385, 386, 387, -1, 389, - 390, 391, 392, -1, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - -1, 411, 412, -1, 414, -1, 416, 417, 418, 419, - 420, -1, 422, 423, 424, -1, -1, 427, 428, 429, - 430, 431, -1, 433, 434, 435, 436, 437, 438, 439, - 440, -1, -1, 443, 444, 445, -1, 447, 448, 449, - 450, -1, 452, 453, 454, 455, 456, 457, 458, -1, - 460, -1, 462, 463, 464, 465, 466, 467, 468, -1, - -1, 471, -1, -1, 474, 475, 476, 477, 478, 479, - 480, 481, 482, 483, 484, 485, 486, 487, 488, 489, - 490, 491, 492, 493, 494, 3, -1, -1, -1, -1, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 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, @@ -17600,7 +17154,7 @@ static const yytype_int16 yycheck[] = 88, -1, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, 156, 157, @@ -17619,220 +17173,220 @@ static const yytype_int16 yycheck[] = 278, -1, 280, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, 319, -1, 321, 322, 323, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, -1, 361, 362, 363, 364, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, -1, - 378, 379, 380, 381, 382, 383, 384, 385, 386, 387, - -1, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, 320, -1, 322, 323, 324, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, -1, 362, 363, 364, 365, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, -1, 381, 382, 383, 384, 385, 386, 387, + 388, 389, -1, -1, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - 408, 409, -1, 411, 412, -1, 414, -1, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, 429, 430, 431, -1, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, 457, - 458, -1, 460, -1, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, + 408, 409, 410, 411, -1, -1, 414, 415, -1, 417, + -1, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, 432, 433, 434, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, 460, 461, -1, 463, -1, 465, 466, 467, + 468, 469, 470, 471, -1, -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, 483, 484, 485, 486, 487, - 488, 489, 490, 491, 492, 493, 494, 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, -1, 75, - 76, 77, 78, 79, -1, -1, -1, 83, 84, 85, - 86, 87, 88, -1, 90, 91, 92, -1, 94, 95, - 96, 97, 98, 99, -1, -1, 102, 103, 104, 105, - 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, - 116, -1, 118, -1, 120, 121, 122, 123, 124, 125, - -1, 127, 128, 129, 130, 131, -1, -1, 134, 135, - 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, - 146, -1, 148, 149, 150, 151, 152, 153, 154, 155, - 156, 157, 158, -1, 160, -1, 162, 163, 164, 165, - -1, 167, -1, 169, -1, -1, -1, 173, 174, 175, - -1, 177, -1, 179, -1, 181, 182, 183, 184, -1, - 186, 187, 188, 189, 190, 191, 192, -1, 194, 195, - 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, - -1, 207, -1, 209, 210, 211, 212, 213, 214, 215, - 216, -1, 218, -1, 220, -1, -1, 223, -1, 225, - 226, 227, 228, 229, 230, -1, -1, 233, -1, 235, - -1, -1, 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, -1, 275, - 276, -1, 278, -1, 280, 281, 282, 283, 284, 285, - -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, - 296, 297, 298, -1, 300, -1, 302, 303, 304, 305, - 306, 307, 308, 309, 310, 311, 312, -1, -1, -1, - -1, 317, 318, 319, -1, 321, 322, 323, 324, 325, - 326, -1, 328, 329, 330, 331, 332, 333, -1, 335, - 336, 337, 338, 339, 340, 341, 342, -1, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, -1, 358, 359, -1, 361, 362, 363, 364, 365, - 366, 367, 368, 369, 370, 371, 372, 373, 374, 375, - -1, -1, 378, 379, 380, 381, 382, 383, 384, 385, - 386, 387, -1, 389, 390, 391, 392, -1, 394, 395, - 396, 397, 398, 399, 400, 401, 402, 403, 404, 405, - 406, 407, 408, 409, -1, 411, 412, -1, 414, -1, - 416, 417, 418, 419, 420, -1, 422, 423, 424, -1, - -1, 427, 428, 429, 430, 431, -1, 433, 434, 435, - 436, 437, 438, 439, 440, -1, -1, 443, 444, 445, - -1, 447, 448, 449, 450, -1, 452, 453, 454, 455, - 456, 457, 458, -1, 460, -1, 462, 463, 464, 465, - 466, 467, 468, -1, -1, 471, -1, -1, 474, 475, - 476, 477, 478, 479, 480, 481, 482, 483, 484, 485, - 486, 487, 488, 489, 490, 491, 492, 493, 494, 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, - -1, 75, 76, 77, 78, 79, -1, -1, -1, 83, - 84, 85, 86, 87, 88, -1, 90, 91, 92, -1, - 94, 95, 96, 97, 98, 99, -1, -1, 102, 103, - 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, - 114, 115, 116, -1, 118, -1, 120, 121, 122, 123, - 124, 125, -1, 127, 128, 129, 130, 131, -1, -1, - 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, - 144, 145, 146, -1, 148, 149, 150, 151, 152, 153, - 154, 155, 156, 157, 158, -1, 160, -1, 162, 163, - 164, 165, -1, 167, -1, 169, -1, -1, -1, 173, - 174, 175, -1, 177, -1, 179, -1, 181, 182, 183, - 184, -1, 186, 187, 188, 189, 190, 191, 192, -1, - 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, - 204, 205, -1, 207, -1, 209, 210, 211, 212, 213, - 214, 215, 216, -1, 218, -1, 220, -1, -1, 223, - -1, 225, 226, 227, 228, 229, 230, -1, -1, 233, - -1, 235, -1, -1, 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, - -1, 275, 276, -1, 278, -1, 280, 281, 282, 283, - 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, - -1, -1, 296, 297, 298, -1, 300, -1, 302, 303, - 304, 305, 306, 307, 308, 309, 310, 311, 312, -1, - -1, -1, -1, 317, 318, 319, -1, 321, 322, 323, - 324, 325, 326, -1, 328, 329, 330, 331, 332, 333, - -1, 335, 336, 337, 338, 339, 340, 341, 342, -1, - 344, 345, 346, 347, 348, 349, 350, 351, 352, 353, - 354, 355, 356, -1, 358, 359, -1, 361, 362, 363, - 364, 365, 366, 367, 368, 369, 370, 371, 372, 373, - 374, 375, -1, -1, 378, 379, 380, 381, 382, 383, - 384, 385, 386, 387, -1, 389, 390, 391, 392, -1, - 394, 395, 396, 397, 398, 399, 400, 401, 402, 403, - 404, 405, 406, 407, 408, 409, -1, 411, 412, -1, - 414, -1, 416, 417, 418, 419, 420, -1, 422, 423, - 424, -1, -1, 427, 428, 429, 430, 431, -1, 433, - 434, 435, 436, 437, 438, 439, 440, -1, -1, 443, - 444, 445, -1, 447, 448, 449, 450, -1, 452, 453, - 454, 455, 456, 457, 458, -1, 460, -1, 462, 463, - 464, 465, 466, 467, 468, -1, -1, 471, -1, -1, - 474, 475, 476, 477, 478, 479, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 491, 492, 493, - 494, 3, -1, -1, -1, -1, -1, -1, -1, -1, + 488, 489, 490, 491, 492, 493, 494, 495, 496, 497, + 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, 57, -1, 59, 60, 61, 62, + 63, 64, -1, -1, 67, 68, 69, 70, 71, 72, + 73, -1, 75, 76, 77, 78, -1, -1, 81, -1, + 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, -1, + 153, 154, 155, 156, 157, -1, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, + 173, 174, -1, 176, 177, -1, 179, -1, -1, -1, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, 208, -1, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, + 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, + 233, 234, 235, 236, -1, 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, -1, 275, 276, 277, 278, -1, -1, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, -1, 299, 300, 301, -1, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, -1, 321, 322, + 323, -1, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, 380, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, -1, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, -1, + 433, -1, -1, 436, 437, 438, 439, 440, 441, 442, + 443, 444, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, -1, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 494, 495, 496, 497, -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, 72, + 73, -1, 75, 76, 77, 78, -1, -1, 81, -1, + 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, -1, + 153, 154, 155, 156, 157, -1, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, + 173, 174, -1, 176, 177, -1, 179, -1, -1, -1, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, 208, -1, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, + 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, + 233, 234, 235, 236, -1, 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, -1, 275, 276, 277, 278, -1, -1, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, -1, 299, 300, 301, -1, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, -1, 321, 322, + 323, -1, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, + 363, 364, 365, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, 380, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, -1, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, -1, + 433, -1, -1, 436, 437, 438, 439, 440, 441, 442, + 443, 444, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, -1, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 494, 495, 496, 497, -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, 72, + 73, -1, 75, 76, 77, 78, -1, -1, 81, -1, + 83, 84, 85, 86, 87, 88, 89, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, -1, + 153, 154, 155, 156, 157, -1, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, 170, -1, 172, + 173, 174, 175, 176, 177, -1, 179, -1, -1, -1, + 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, + 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, 208, -1, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, + 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, + 233, 234, 235, 236, -1, 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, -1, 275, 276, 277, 278, -1, -1, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, -1, 299, 300, 301, -1, + 303, 304, 305, 306, 307, 308, 309, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, -1, 321, 322, + 323, -1, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, 337, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, 361, 362, + 363, 364, -1, 366, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, 380, 381, 382, + 383, 384, 385, 386, -1, 388, 389, -1, 391, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, -1, -1, + -1, 414, 415, -1, 417, 418, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, -1, + 433, -1, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, -1, 461, -1, + 463, 464, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 3, -1, -1, -1, -1, -1, -1, -1, -1, -1, + -1, 494, 495, 496, 497, -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, -1, 75, 76, 77, 78, 79, -1, -1, -1, + 83, 84, 85, 86, 87, 88, -1, 90, 91, 92, + -1, 94, 95, 96, 97, 98, 99, -1, -1, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, -1, 118, -1, 120, 121, 122, + 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, + -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, + -1, 144, 145, 146, -1, 148, 149, 150, 151, 152, + 153, 154, 155, 156, 157, 158, -1, 160, -1, 162, + 163, 164, 165, -1, 167, -1, 169, -1, -1, -1, + 173, 174, 175, -1, 177, -1, 179, -1, 181, 182, + -1, 184, -1, 186, 187, 188, 189, 190, 191, 192, + -1, 194, 195, 196, 197, -1, 199, 200, 201, 202, + 203, 204, 205, -1, 207, -1, 209, 210, 211, 212, + 213, 214, 215, 216, -1, 218, -1, 220, -1, -1, + 223, -1, 225, 226, 227, 228, 229, 230, -1, -1, + 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, 281, 282, + 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, + 293, -1, -1, 296, 297, 298, -1, 300, -1, 302, + 303, 304, 305, 306, 307, 308, -1, 310, 311, 312, + 313, -1, -1, -1, -1, 318, 319, 320, -1, 322, + 323, 324, 325, 326, 327, -1, 329, 330, 331, 332, + 333, 334, -1, 336, -1, 338, 339, 340, 341, 342, + 343, -1, 345, 346, 347, 348, 349, 350, 351, 352, + 353, 354, 355, 356, 357, -1, 359, 360, -1, 362, + 363, 364, 365, -1, 367, 368, 369, 370, 371, 372, + 373, 374, 375, 376, 377, 378, -1, -1, 381, 382, + 383, 384, 385, 386, 387, 388, 389, -1, -1, 392, + 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, + 403, 404, 405, 406, 407, 408, 409, 410, 411, -1, + -1, 414, 415, -1, 417, -1, 419, 420, 421, 422, + 423, -1, 425, 426, 427, -1, -1, 430, 431, 432, + 433, 434, -1, 436, 437, 438, 439, 440, 441, 442, + 443, -1, -1, 446, 447, 448, -1, 450, 451, 452, + 453, -1, 455, 456, 457, 458, 459, 460, 461, -1, + 463, -1, 465, 466, 467, 468, 469, 470, 471, -1, + -1, 474, -1, -1, 477, 478, 479, 480, 481, 482, + 483, 484, 485, 486, 487, 488, 489, 490, 491, 492, + 493, 494, 495, 496, 497, 3, -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, - 72, 73, -1, 75, 76, 77, 78, -1, -1, 81, - -1, 83, 84, 85, 86, 87, 88, 89, 90, 91, - 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, - 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, - 112, 113, 114, 115, 116, -1, 118, -1, 120, 121, - 122, 123, 124, 125, -1, 127, 128, 129, 130, 131, - -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, - 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, - -1, 153, 154, 155, 156, 157, -1, -1, 160, -1, - 162, 163, 164, 165, -1, 167, -1, 169, 170, -1, - 172, 173, 174, -1, 176, 177, -1, 179, -1, -1, - -1, 183, 184, -1, 186, 187, 188, 189, 190, 191, - 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, - 202, 203, 204, 205, -1, 207, 208, -1, 210, 211, - 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, - 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, - -1, 233, 234, 235, 236, -1, 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, -1, 275, 276, 277, 278, -1, -1, 281, - 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, - 292, 293, -1, -1, 296, 297, -1, 299, 300, 301, - -1, 303, 304, 305, 306, 307, 308, 309, 310, 311, - 312, -1, -1, -1, -1, 317, 318, -1, 320, 321, - 322, -1, 324, 325, 326, -1, 328, 329, 330, 331, - 332, 333, -1, 335, 336, 337, 338, 339, 340, 341, - 342, -1, 344, 345, 346, 347, 348, 349, 350, 351, - 352, 353, 354, 355, 356, -1, 358, 359, 360, 361, - 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, - 372, 373, 374, 375, -1, 377, 378, 379, 380, 381, - 382, 383, 384, 385, 386, 387, 388, 389, 390, 391, - 392, -1, 394, 395, 396, 397, 398, 399, 400, 401, - 402, 403, 404, 405, 406, 407, -1, 409, -1, 411, - 412, -1, 414, 415, 416, 417, 418, 419, 420, -1, - 422, 423, 424, -1, -1, 427, 428, -1, 430, -1, - -1, 433, 434, 435, 436, 437, 438, 439, 440, 441, - -1, 443, 444, 445, -1, 447, 448, 449, 450, -1, - 452, 453, 454, 455, 456, -1, 458, -1, 460, 461, - 462, 463, 464, 465, 466, 467, 468, -1, -1, 471, - -1, -1, 474, 475, 476, 477, 478, 479, 3, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, 491, - 492, 493, 494, -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, 72, 73, -1, - 75, 76, 77, 78, -1, -1, 81, -1, 83, 84, - 85, 86, 87, 88, 89, 90, 91, 92, -1, 94, - 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, - 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, - 115, 116, -1, 118, -1, 120, 121, 122, 123, 124, - 125, -1, 127, 128, 129, 130, 131, -1, -1, 134, - 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, - 145, 146, -1, 148, 149, 150, 151, -1, 153, 154, - 155, 156, 157, -1, -1, 160, -1, 162, 163, 164, - 165, -1, 167, -1, 169, 170, -1, 172, 173, 174, - -1, 176, 177, -1, 179, -1, -1, -1, 183, 184, - -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, - 205, -1, 207, 208, -1, 210, 211, 212, 213, 214, - 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, - 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, - 235, 236, -1, 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, -1, - 275, 276, 277, 278, -1, -1, 281, 282, 283, 284, - 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, - -1, 296, 297, -1, 299, 300, 301, -1, 303, 304, - 305, 306, 307, 308, 309, 310, 311, 312, -1, -1, - -1, -1, 317, 318, -1, 320, 321, 322, -1, 324, - 325, 326, -1, 328, 329, 330, 331, 332, 333, -1, - 335, 336, 337, 338, 339, 340, 341, 342, -1, 344, - 345, 346, 347, 348, 349, 350, 351, 352, 353, 354, - 355, 356, -1, 358, 359, 360, 361, 362, 363, 364, - 365, 366, 367, 368, 369, 370, 371, 372, 373, 374, - 375, -1, 377, 378, 379, 380, 381, 382, 383, 384, - 385, 386, 387, 388, 389, 390, 391, 392, -1, 394, - 395, 396, 397, 398, 399, 400, 401, 402, 403, 404, - 405, 406, 407, -1, 409, -1, 411, 412, -1, 414, - 415, 416, 417, 418, 419, 420, -1, 422, 423, 424, - -1, -1, 427, 428, -1, 430, -1, -1, 433, 434, - 435, 436, 437, 438, 439, 440, 441, -1, 443, 444, - 445, -1, 447, 448, 449, 450, -1, 452, 453, 454, - 455, 456, -1, 458, -1, 460, 461, 462, 463, 464, - 465, 466, 467, 468, -1, -1, 471, -1, -1, 474, - 475, 476, 477, 478, 479, 3, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 491, 492, 493, 494, -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, @@ -17843,631 +17397,949 @@ static const yytype_int16 yycheck[] = 88, 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, -1, - 118, -1, 120, 121, 122, 123, 124, 125, -1, 127, + 118, -1, 120, 121, 122, 123, 124, 125, -1, -1, 128, 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, 151, -1, 153, 154, 155, 156, 157, -1, -1, 160, -1, 162, 163, 164, 165, -1, 167, - -1, 169, 170, -1, 172, 173, 174, 175, 176, 177, + -1, 169, 170, -1, 172, 173, 174, -1, 176, 177, -1, 179, -1, -1, -1, 183, 184, -1, 186, 187, 188, 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, -1, 210, 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, 229, 230, -1, -1, 233, 234, 235, 236, -1, - 238, 239, 240, 241, 242, 243, 244, 245, 246, 247, + 238, 239, 240, 241, 242, 243, 244, 245, -1, 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, -1, 275, 276, 277, 278, -1, -1, 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, 291, 292, 293, -1, -1, 296, 297, -1, 299, 300, 301, -1, 303, 304, 305, 306, 307, - 308, 309, 310, 311, 312, -1, -1, -1, -1, 317, - 318, -1, 320, 321, 322, -1, 324, 325, 326, -1, - 328, 329, 330, 331, 332, 333, -1, 335, 336, 337, - 338, 339, 340, 341, 342, -1, 344, 345, 346, 347, - 348, 349, 350, 351, 352, 353, 354, 355, 356, -1, - 358, 359, 360, 361, 362, 363, -1, 365, 366, 367, - 368, 369, 370, 371, 372, 373, 374, 375, -1, 377, - 378, 379, 380, 381, 382, 383, -1, 385, 386, 387, - 388, 389, 390, 391, 392, -1, 394, 395, 396, 397, + 308, 309, 310, 311, 312, 313, -1, -1, -1, -1, + 318, 319, -1, 321, 322, 323, -1, 325, 326, 327, + -1, 329, 330, 331, 332, 333, 334, -1, 336, 337, + 338, 339, 340, 341, 342, 343, -1, 345, 346, 347, + 348, 349, 350, 351, 352, 353, 354, 355, 356, 357, + -1, 359, 360, 361, 362, 363, 364, -1, 366, 367, + 368, 369, 370, 371, 372, 373, 374, 375, 376, 377, + 378, -1, 380, 381, 382, 383, 384, 385, 386, -1, + 388, 389, -1, 391, 392, 393, 394, 395, -1, 397, 398, 399, 400, 401, 402, 403, 404, 405, 406, 407, - -1, 409, -1, 411, 412, -1, 414, 415, 416, 417, - 418, 419, 420, -1, 422, 423, 424, -1, -1, 427, - 428, -1, 430, -1, -1, 433, 434, 435, 436, 437, - 438, 439, 440, -1, -1, 443, 444, 445, -1, 447, - 448, 449, 450, -1, 452, 453, 454, 455, 456, -1, - 458, -1, 460, 461, 462, 463, 464, 465, 466, 467, - 468, -1, -1, 471, -1, -1, 474, 475, 476, 477, - 478, 479, 3, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, 491, 492, 493, 494, -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, -1, 75, 76, 77, 78, 79, -1, - -1, -1, 83, 84, 85, 86, 87, 88, -1, 90, - 91, 92, -1, 94, 95, 96, 97, 98, 99, -1, - -1, 102, 103, 104, 105, 106, 107, 108, 109, 110, - 111, 112, 113, 114, 115, 116, -1, 118, -1, 120, - 121, 122, 123, 124, 125, -1, 127, 128, 129, 130, - 131, -1, -1, 134, 135, 136, 137, 138, -1, 140, - 141, 142, -1, 144, 145, 146, -1, 148, 149, 150, - 151, 152, 153, 154, 155, 156, 157, 158, -1, 160, - -1, 162, 163, 164, 165, -1, 167, -1, 169, -1, - -1, -1, 173, 174, 175, -1, 177, -1, 179, -1, - 181, 182, -1, 184, -1, 186, 187, 188, 189, 190, - 191, 192, -1, 194, 195, 196, 197, -1, 199, 200, - 201, 202, 203, 204, 205, -1, 207, -1, 209, 210, - 211, 212, 213, 214, 215, 216, -1, 218, -1, 220, - -1, -1, 223, -1, 225, 226, 227, 228, 229, 230, - -1, -1, 233, -1, 235, -1, -1, 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, -1, 275, 276, -1, 278, -1, 280, - 281, 282, 283, 284, 285, -1, 287, 288, -1, -1, - 291, 292, 293, -1, -1, 296, 297, 298, -1, 300, - -1, 302, 303, 304, 305, 306, 307, 308, -1, 310, - 311, 312, -1, -1, -1, -1, 317, 318, 319, -1, - 321, 322, 323, 324, 325, 326, -1, 328, 329, 330, - 331, 332, 333, -1, 335, -1, 337, 338, 339, 340, - 341, 342, -1, 344, 345, 346, 347, 348, 349, 350, - 351, 352, 353, 354, 355, 356, -1, 358, 359, -1, - 361, 362, 363, 364, -1, 366, 367, 368, 369, 370, - 371, 372, 373, 374, 375, -1, -1, 378, 379, 380, - 381, 382, 383, 384, 385, 386, 387, -1, 389, 390, - 391, 392, -1, 394, 395, 396, 397, 398, 399, 400, - 401, 402, 403, 404, 405, 406, 407, 408, 409, -1, - 411, 412, -1, 414, -1, 416, 417, 418, 419, 420, - -1, 422, 423, 424, -1, -1, 427, 428, 429, 430, - 431, -1, 433, 434, 435, 436, 437, 438, 439, 440, - -1, -1, 443, 444, 445, -1, 447, 448, 449, 450, - -1, 452, 453, 454, 455, 456, 457, 458, -1, 460, - -1, 462, 463, 464, 465, 466, 467, 468, -1, -1, - 471, -1, -1, 474, 475, 476, 477, 478, 479, 480, - 481, 482, 483, 484, 485, 486, 487, 488, 489, 490, - 491, 492, 493, 494, 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, 57, -1, - 59, 60, 61, 62, 63, 64, -1, -1, 67, 68, - 69, 70, 71, 72, 73, -1, 75, 76, 77, 78, - -1, -1, 81, -1, 83, 84, 85, 86, 87, 88, - 89, 90, 91, 92, -1, 94, 95, 96, 97, 98, - 99, -1, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, -1, 118, - -1, 120, 121, 122, 123, 124, 125, -1, 127, 128, - 129, 130, 131, -1, -1, 134, 135, 136, 137, 138, - -1, 140, 141, 142, -1, 144, 145, 146, -1, 148, - 149, 150, 151, -1, 153, 154, 155, 156, 157, -1, - -1, 160, -1, 162, 163, 164, 165, -1, 167, -1, - 169, 170, -1, 172, 173, 174, -1, 176, 177, -1, - 179, -1, -1, -1, 183, 184, -1, 186, 187, 188, - 189, 190, 191, 192, 193, 194, 195, 196, 197, -1, - 199, 200, 201, 202, 203, 204, 205, -1, 207, 208, - -1, 210, 211, 212, 213, 214, 215, 216, -1, 218, - -1, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, -1, -1, 233, 234, 235, 236, -1, 238, - 239, 240, 241, 242, 243, 244, 245, -1, 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, -1, 275, 276, 277, 278, - -1, -1, 281, 282, 283, 284, 285, -1, 287, 288, - -1, -1, 291, 292, 293, -1, -1, 296, 297, -1, - 299, 300, 301, -1, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, -1, -1, -1, -1, 317, 318, - -1, 320, 321, 322, -1, 324, 325, 326, -1, 328, - 329, 330, 331, 332, 333, -1, 335, 336, 337, 338, - 339, 340, 341, 342, -1, 344, 345, 346, 347, 348, - 349, 350, 351, 352, 353, 354, 355, 356, -1, 358, - 359, 360, 361, 362, 363, -1, 365, 366, 367, 368, - 369, 370, 371, 372, 373, 374, 375, -1, 377, 378, - 379, 380, 381, 382, 383, -1, 385, 386, 387, 388, - 389, 390, 391, 392, -1, 394, 395, 396, 397, 398, - 399, 400, 401, 402, 403, 404, 405, -1, 407, -1, - 409, -1, 411, 412, -1, 414, 415, 416, 417, 418, - 419, 420, -1, 422, 423, 424, -1, -1, 427, 428, - -1, 430, -1, -1, 433, 434, 435, 436, 437, 438, - 439, 440, -1, -1, 443, 444, 445, -1, 447, 448, - 449, 450, -1, 452, 453, 454, 455, 456, -1, 458, - -1, 460, 461, 462, 463, 464, 465, 466, 467, 468, - -1, 22, 471, -1, -1, 474, 475, 476, 477, 478, - 479, 32, -1, 34, 35, -1, -1, -1, -1, -1, - -1, -1, 491, 492, 493, 494, -1, -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, -1, -1, 75, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 86, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, 98, -1, 100, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 111, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 127, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 137, -1, -1, -1, - -1, -1, 143, -1, -1, -1, -1, -1, -1, -1, - 151, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 167, -1, -1, -1, - 171, -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, + 408, -1, 410, -1, -1, -1, 414, 415, -1, 417, + 418, 419, 420, 421, 422, 423, -1, 425, 426, 427, + -1, -1, 430, 431, -1, 433, -1, -1, 436, 437, + 438, 439, 440, 441, 442, 443, -1, -1, 446, 447, + 448, -1, 450, 451, 452, 453, -1, 455, 456, 457, + 458, 459, -1, 461, -1, 463, 464, 465, 466, 467, + 468, 469, 470, 471, -1, 22, 474, -1, -1, 477, + 478, 479, 480, 481, 482, 32, -1, 34, 35, -1, + -1, -1, -1, 22, -1, -1, 494, 495, 496, 497, + -1, -1, -1, 32, -1, 52, -1, -1, -1, -1, + -1, -1, -1, -1, 61, -1, -1, -1, -1, -1, + -1, -1, -1, 52, -1, -1, -1, -1, 75, -1, + -1, -1, 61, -1, -1, -1, -1, -1, -1, 86, + -1, -1, -1, -1, -1, -1, 75, -1, -1, -1, + -1, 98, -1, 100, -1, -1, -1, 86, -1, -1, + -1, -1, -1, -1, 111, -1, -1, -1, -1, 98, + -1, 100, -1, -1, -1, -1, -1, -1, -1, 126, + 127, -1, 111, -1, -1, -1, -1, -1, -1, -1, + 137, -1, -1, -1, -1, -1, 143, 126, 127, -1, + -1, -1, -1, -1, 151, -1, -1, -1, 137, -1, + -1, -1, -1, -1, 143, -1, -1, -1, -1, -1, + 167, -1, 151, -1, 171, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, -1, 167, -1, + -1, -1, 171, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 213, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, 213, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 239, -1, + -1, -1, -1, -1, 213, -1, -1, -1, -1, -1, + -1, -1, 239, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, + 239, -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, 314, 315, 316, + -1, -1, -1, -1, -1, 322, -1, -1, 325, -1, + -1, -1, -1, -1, -1, 314, 315, 316, -1, -1, + -1, -1, -1, 322, -1, -1, 325, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 354, -1, -1, + -1, -1, -1, -1, -1, -1, 363, -1, -1, -1, + -1, -1, -1, -1, -1, 354, -1, -1, -1, -1, + -1, -1, 379, -1, 363, -1, -1, -1, -1, 386, + -1, -1, -1, 390, -1, -1, -1, -1, -1, -1, + 379, -1, -1, 400, -1, -1, -1, 386, -1, -1, + -1, 390, -1, -1, -1, 412, -1, -1, -1, 416, + -1, 400, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, 412, -1, -1, -1, 416, -1, 436, -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 313, 314, 315, -1, -1, -1, -1, -1, - 321, -1, -1, 324, -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, 353, -1, -1, -1, -1, -1, -1, -1, - -1, 362, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 376, -1, -1, -1, -1, - -1, -1, 383, -1, -1, -1, 387, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 397, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, 409, -1, - -1, -1, 413, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 433, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, 446, -1, -1, -1, -1, - -1, 452, -1, -1, -1, -1, 457, -1, -1, -1, - 461, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, 473, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - -1, -1, -1, -1, -1, -1, 497, -1, -1, -1, - -1, -1, -1, -1, -1, -1, -1, -1, -1, -1, - 511 + -1, -1, 449, -1, -1, -1, -1, 436, 455, -1, + -1, -1, -1, 460, -1, -1, -1, 464, -1, -1, + 449, -1, -1, -1, -1, -1, 455, -1, -1, 476, + -1, 460, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 476, -1, -1, + -1, -1, -1, 500, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, -1, -1, 514, -1, -1, + -1, 500, -1, -1, -1, -1, -1, -1, -1, -1, + -1, -1, -1, -1, -1, 514 }; -/* YYSTOS[STATE-NUM] -- The (internal number of the) accessing - symbol of state STATE-NUM. */ -static const yytype_uint16 yystos[] = +/* YYSTOS[STATE-NUM] -- The symbol kind of the accessing symbol of + state STATE-NUM. */ +static const yytype_int16 yystos[] = { - 0, 22, 32, 34, 35, 47, 52, 61, 75, 86, - 98, 100, 111, 127, 128, 137, 143, 151, 153, 154, - 167, 171, 197, 213, 239, 313, 314, 315, 321, 324, - 353, 362, 376, 383, 387, 397, 409, 413, 433, 446, - 449, 452, 457, 473, 497, 511, 523, 524, 525, 526, - 537, 546, 548, 553, 569, 572, 573, 575, 579, 586, - 588, 589, 637, 643, 646, 647, 665, 666, 667, 668, - 670, 672, 673, 677, 730, 731, 897, 900, 903, 910, - 911, 913, 914, 915, 922, 926, 932, 935, 940, 944, - 945, 946, 949, 952, 953, 957, 958, 960, 427, 476, - 587, 202, 369, 378, 413, 463, 108, 191, 947, 587, - 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, 63, 64, 67, 68, 69, 70, 71, 72, 73, - 75, 76, 77, 78, 79, 81, 83, 84, 85, 86, - 87, 88, 89, 90, 91, 92, 94, 95, 96, 97, - 98, 99, 101, 102, 103, 104, 105, 106, 107, 108, - 109, 110, 111, 112, 113, 114, 115, 116, 118, 120, - 121, 122, 123, 124, 125, 127, 128, 129, 130, 131, - 134, 135, 136, 137, 138, 140, 141, 142, 144, 145, - 146, 148, 149, 150, 151, 152, 153, 154, 155, 156, - 157, 158, 160, 162, 163, 164, 165, 167, 169, 170, - 172, 173, 174, 175, 176, 177, 179, 181, 182, 183, - 184, 186, 187, 188, 189, 190, 191, 192, 193, 194, - 195, 196, 197, 199, 200, 201, 202, 203, 204, 205, - 207, 208, 209, 210, 211, 212, 213, 214, 215, 216, - 218, 220, 221, 222, 223, 224, 225, 226, 227, 228, - 229, 230, 233, 234, 235, 236, 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, 275, 276, 277, 278, 280, 281, 282, 283, - 284, 285, 287, 288, 291, 292, 293, 296, 297, 298, - 299, 300, 301, 302, 303, 304, 305, 306, 307, 308, - 309, 310, 311, 312, 317, 318, 319, 320, 321, 322, - 323, 324, 325, 326, 328, 329, 330, 331, 332, 333, - 335, 336, 337, 338, 339, 340, 341, 342, 344, 345, - 346, 347, 348, 349, 350, 351, 352, 353, 354, 355, - 356, 358, 359, 360, 361, 362, 363, 364, 365, 366, - 367, 368, 369, 370, 371, 372, 373, 374, 375, 378, - 379, 380, 381, 382, 383, 384, 385, 386, 387, 388, - 389, 390, 391, 392, 394, 395, 396, 397, 398, 399, - 400, 401, 402, 403, 404, 405, 406, 407, 408, 409, - 411, 412, 414, 415, 416, 417, 418, 419, 420, 422, - 423, 424, 427, 428, 429, 430, 431, 433, 434, 435, - 436, 437, 438, 439, 440, 443, 444, 445, 447, 448, - 449, 450, 452, 453, 454, 455, 456, 457, 458, 460, - 461, 462, 463, 464, 465, 466, 467, 468, 471, 474, - 475, 476, 477, 478, 479, 480, 481, 482, 483, 484, - 485, 486, 487, 488, 489, 490, 491, 492, 493, 494, - 539, 806, 886, 890, 963, 964, 965, 3, 175, 246, - 406, 539, 912, 963, 587, 55, 171, 511, 660, 177, - 240, 294, 369, 417, 419, 436, 442, 445, 635, 909, - 5, 30, 324, 539, 540, 885, 108, 947, 24, 81, - 97, 146, 156, 168, 173, 202, 245, 249, 318, 333, - 366, 369, 378, 381, 399, 413, 420, 430, 436, 463, - 638, 639, 642, 587, 885, 461, 511, 526, 537, 546, - 548, 569, 572, 573, 575, 579, 586, 589, 637, 643, - 646, 647, 665, 897, 900, 903, 910, 911, 918, 922, - 926, 932, 935, 940, 949, 952, 957, 958, 960, 108, - 75, 213, 66, 79, 81, 158, 231, 280, 290, 302, - 319, 365, 408, 429, 431, 435, 457, 511, 538, 539, - 540, 666, 731, 733, 735, 745, 752, 753, 806, 808, - 809, 108, 5, 539, 541, 933, 933, 539, 885, 30, - 177, 240, 382, 423, 427, 539, 950, 951, 955, 587, - 30, 132, 686, 687, 177, 240, 369, 382, 423, 927, - 928, 955, 587, 539, 665, 677, 956, 539, 752, 413, - 683, 538, 172, 511, 937, 511, 341, 678, 679, 885, - 678, 666, 667, 0, 514, 122, 212, 448, 147, 217, - 295, 441, 689, 690, 735, 735, 666, 668, 670, 515, - 461, 916, 30, 423, 427, 665, 956, 191, 538, 885, - 191, 538, 191, 752, 191, 538, 274, 541, 509, 513, - 542, 543, 511, 108, 665, 538, 417, 419, 417, 419, - 351, 191, 538, 538, 173, 245, 341, 378, 413, 463, - 644, 202, 30, 885, 191, 3, 251, 430, 107, 413, - 413, 463, 372, 191, 539, 640, 892, 191, 882, 885, - 191, 885, 511, 578, 918, 3, 37, 42, 49, 53, - 54, 55, 56, 57, 71, 72, 79, 81, 83, 89, - 101, 112, 115, 152, 158, 164, 170, 172, 175, 176, - 181, 182, 193, 208, 209, 215, 216, 218, 221, 222, - 224, 234, 236, 246, 267, 268, 269, 273, 277, 280, - 282, 298, 299, 301, 302, 319, 320, 323, 338, 360, - 364, 377, 384, 388, 391, 406, 408, 415, 423, 424, - 429, 431, 435, 457, 458, 461, 480, 481, 482, 483, - 484, 485, 486, 487, 488, 489, 490, 919, 921, 922, - 924, 925, 963, 967, 916, 539, 541, 912, 933, 511, - 511, 166, 511, 666, 753, 511, 511, 538, 511, 511, - 171, 511, 511, 511, 511, 666, 731, 735, 745, 504, - 542, 40, 539, 746, 747, 746, 376, 515, 669, 37, - 42, 101, 172, 208, 224, 234, 268, 313, 320, 360, - 377, 446, 749, 747, 40, 539, 746, 748, 497, 757, - 541, 171, 500, 511, 511, 898, 951, 951, 951, 494, - 223, 513, 289, 4, 6, 7, 8, 9, 10, 39, - 54, 56, 57, 65, 71, 72, 83, 112, 115, 117, - 136, 152, 159, 164, 181, 182, 215, 216, 218, 246, - 267, 269, 274, 279, 282, 291, 338, 364, 391, 423, - 424, 432, 458, 495, 502, 503, 504, 509, 511, 516, - 517, 518, 519, 539, 541, 666, 720, 769, 772, 775, - 776, 777, 779, 780, 781, 782, 784, 785, 800, 802, - 803, 804, 805, 806, 807, 809, 810, 825, 826, 837, - 859, 864, 872, 873, 874, 886, 887, 888, 871, 873, - 927, 927, 541, 927, 494, 171, 425, 500, 513, 542, - 752, 941, 3, 170, 172, 461, 922, 936, 938, 170, - 939, 800, 843, 844, 678, 515, 511, 894, 512, 512, - 525, 171, 294, 556, 941, 30, 132, 684, 684, 59, - 684, 161, 166, 237, 286, 695, 697, 698, 723, 725, - 726, 727, 180, 289, 451, 289, 689, 690, 511, 538, - 414, 954, 494, 223, 152, 26, 32, 137, 293, 349, - 353, 383, 454, 531, 534, 535, 349, 152, 40, 60, - 106, 201, 250, 260, 272, 304, 349, 355, 378, 383, - 397, 534, 580, 583, 152, 349, 383, 534, 152, 349, - 383, 534, 152, 40, 948, 800, 865, 3, 30, 34, - 35, 36, 38, 39, 40, 41, 45, 58, 65, 66, - 74, 80, 82, 93, 100, 117, 119, 126, 132, 133, - 139, 143, 147, 159, 161, 166, 168, 171, 178, 180, - 185, 198, 206, 217, 219, 231, 232, 237, 274, 279, - 286, 289, 290, 294, 295, 313, 314, 315, 316, 327, - 334, 343, 357, 376, 393, 410, 413, 421, 425, 426, - 432, 441, 442, 446, 451, 457, 459, 469, 470, 472, - 473, 544, 545, 963, 967, 969, 543, 3, 30, 37, - 42, 49, 55, 81, 83, 89, 101, 132, 170, 172, - 175, 176, 193, 208, 221, 222, 224, 234, 236, 246, - 268, 277, 299, 301, 320, 360, 377, 388, 406, 415, - 435, 459, 461, 512, 800, 846, 847, 889, 895, 963, - 968, 539, 512, 511, 624, 369, 635, 274, 901, 40, - 463, 191, 538, 191, 538, 962, 191, 538, 191, 538, - 89, 906, 152, 477, 90, 129, 307, 418, 152, 513, - 893, 63, 356, 515, 641, 152, 515, 641, 152, 289, - 576, 577, 800, 895, 512, 515, 4, 159, 289, 432, - 502, 503, 541, 582, 585, 888, 917, 919, 920, 923, - 918, 425, 511, 655, 659, 171, 800, 844, 511, 3, - 68, 69, 109, 110, 113, 114, 188, 189, 252, 253, - 254, 255, 256, 257, 258, 259, 262, 263, 373, 374, - 467, 468, 491, 492, 541, 787, 788, 789, 790, 791, - 792, 793, 794, 795, 796, 797, 798, 849, 850, 747, - 748, 800, 538, 800, 851, 502, 503, 539, 801, 802, - 826, 837, 853, 511, 800, 843, 854, 800, 58, 171, - 232, 426, 800, 844, 857, 800, 512, 540, 511, 415, - 703, 704, 704, 686, 687, 735, 219, 681, 224, 37, - 224, 377, 749, 224, 299, 750, 735, 750, 224, 749, - 511, 224, 750, 224, 148, 199, 737, 224, 704, 511, - 540, 511, 704, 296, 539, 541, 934, 800, 929, 931, - 846, 3, 37, 42, 49, 54, 55, 56, 57, 71, - 72, 81, 83, 89, 101, 112, 115, 164, 170, 172, - 176, 193, 208, 215, 216, 218, 221, 222, 224, 234, - 236, 246, 267, 268, 269, 277, 282, 299, 301, 320, - 338, 360, 364, 377, 384, 388, 391, 406, 415, 423, - 424, 435, 441, 458, 461, 763, 765, 766, 768, 770, - 772, 774, 776, 777, 778, 780, 781, 784, 785, 848, - 891, 963, 966, 40, 235, 539, 511, 509, 666, 460, - 783, 800, 863, 783, 783, 511, 511, 771, 771, 323, - 666, 511, 773, 519, 71, 72, 783, 800, 771, 511, - 511, 475, 497, 511, 786, 511, 786, 800, 800, 800, - 148, 875, 876, 800, 844, 845, 666, 800, 843, 9, - 545, 540, 827, 828, 829, 513, 542, 866, 542, 511, - 541, 511, 511, 541, 888, 3, 8, 11, 15, 16, - 17, 18, 19, 20, 21, 36, 40, 46, 53, 80, - 176, 193, 198, 221, 222, 236, 274, 277, 291, 294, - 388, 495, 498, 499, 500, 502, 503, 504, 505, 506, - 507, 835, 836, 837, 839, 869, 474, 811, 301, 800, - 515, 681, 511, 541, 681, 3, 117, 240, 541, 582, - 785, 930, 104, 931, 931, 539, 40, 539, 512, 515, - 916, 515, 512, 679, 882, 883, 40, 941, 192, 351, - 219, 383, 668, 668, 30, 691, 692, 800, 59, 668, - 685, 163, 271, 711, 226, 272, 337, 386, 448, 4, - 9, 30, 706, 800, 502, 503, 707, 708, 800, 802, - 723, 724, 698, 697, 695, 696, 166, 726, 284, 728, - 59, 674, 675, 676, 738, 801, 873, 873, 695, 723, - 844, 894, 235, 538, 74, 82, 93, 168, 191, 327, - 442, 539, 606, 616, 631, 82, 93, 547, 93, 547, - 511, 425, 511, 604, 244, 445, 604, 93, 515, 425, - 538, 3, 768, 582, 59, 584, 582, 582, 106, 250, - 260, 59, 425, 473, 497, 581, 265, 369, 581, 583, - 752, 93, 425, 547, 369, 538, 425, 369, 947, 539, - 655, 510, 521, 846, 846, 847, 515, 689, 690, 13, - 14, 425, 425, 539, 623, 628, 473, 658, 538, 341, - 378, 413, 463, 644, 152, 100, 573, 589, 902, 903, - 958, 144, 765, 538, 274, 574, 580, 274, 511, 624, - 40, 274, 624, 274, 511, 645, 191, 539, 618, 907, - 3, 640, 544, 513, 892, 882, 885, 885, 892, 512, - 515, 13, 918, 924, 4, 888, 4, 888, 541, 545, - 657, 664, 55, 102, 123, 141, 145, 167, 170, 186, - 279, 287, 309, 335, 661, 934, 40, 512, 800, 512, - 171, 515, 512, 316, 852, 512, 801, 801, 11, 15, - 16, 19, 20, 21, 198, 221, 291, 498, 499, 500, - 502, 503, 504, 505, 506, 507, 837, 801, 512, 754, - 755, 808, 166, 171, 855, 856, 515, 512, 40, 857, - 844, 857, 857, 171, 512, 40, 746, 511, 883, 4, - 9, 539, 699, 701, 702, 873, 871, 177, 240, 413, - 417, 419, 445, 538, 682, 470, 758, 735, 735, 224, - 735, 289, 451, 751, 735, 224, 873, 735, 735, 281, - 281, 511, 735, 540, 759, 760, 511, 540, 759, 515, - 512, 515, 513, 511, 768, 511, 511, 513, 39, 767, - 511, 787, 788, 789, 790, 791, 792, 793, 794, 795, - 796, 797, 798, 799, 512, 515, 771, 548, 646, 647, - 665, 899, 940, 844, 845, 511, 469, 860, 861, 800, - 845, 888, 800, 830, 831, 832, 833, 783, 783, 8, - 15, 16, 19, 20, 21, 498, 499, 500, 502, 503, - 504, 505, 506, 507, 539, 835, 840, 512, 844, 423, - 423, 888, 888, 511, 539, 351, 880, 166, 510, 512, - 515, 521, 515, 520, 504, 543, 844, 888, 800, 799, - 799, 765, 800, 800, 800, 800, 800, 800, 800, 800, - 5, 545, 896, 423, 45, 410, 870, 892, 800, 800, - 511, 666, 858, 132, 159, 274, 279, 284, 432, 443, - 800, 279, 511, 800, 425, 53, 176, 193, 198, 236, - 388, 800, 800, 800, 800, 800, 800, 800, 800, 800, - 800, 30, 38, 393, 834, 509, 513, 868, 180, 162, - 812, 364, 511, 826, 874, 171, 732, 846, 732, 511, - 541, 539, 538, 936, 538, 944, 800, 515, 512, 249, - 274, 680, 451, 943, 538, 550, 511, 539, 555, 565, - 566, 568, 41, 126, 693, 515, 451, 693, 265, 668, - 364, 365, 502, 503, 708, 710, 802, 386, 226, 290, - 312, 312, 515, 506, 4, 709, 888, 709, 364, 365, - 710, 538, 881, 278, 390, 729, 511, 883, 884, 515, - 180, 451, 198, 180, 219, 724, 696, 512, 349, 534, - 511, 191, 616, 885, 226, 274, 226, 451, 511, 609, - 764, 765, 885, 539, 191, 885, 191, 539, 26, 137, - 383, 530, 533, 545, 600, 614, 885, 545, 608, 627, - 885, 531, 885, 349, 383, 534, 580, 582, 892, 885, - 582, 892, 885, 582, 349, 383, 534, 885, 885, 885, - 885, 349, 383, 534, 885, 885, 541, 503, 800, 865, - 689, 689, 689, 459, 847, 192, 354, 688, 800, 800, - 539, 332, 654, 512, 515, 287, 171, 425, 649, 901, - 463, 538, 538, 962, 538, 538, 538, 294, 635, 511, - 666, 511, 152, 152, 236, 539, 606, 616, 619, 622, - 632, 634, 473, 475, 611, 151, 665, 152, 473, 908, - 152, 512, 846, 40, 274, 289, 641, 544, 641, 289, - 641, 576, 800, 655, 504, 511, 582, 656, 805, 923, - 512, 515, 40, 653, 541, 653, 274, 279, 335, 653, - 59, 653, 765, 512, 800, 800, 800, 855, 765, 801, - 801, 801, 801, 801, 801, 132, 274, 284, 801, 801, - 801, 801, 801, 801, 801, 801, 801, 801, 512, 515, - 40, 756, 800, 800, 856, 855, 765, 512, 512, 512, - 844, 765, 883, 512, 312, 506, 312, 365, 506, 511, - 511, 681, 417, 419, 417, 419, 538, 683, 683, 683, - 800, 180, 712, 751, 751, 735, 800, 511, 735, 166, - 751, 511, 540, 742, 751, 765, 512, 515, 759, 512, - 929, 3, 848, 39, 767, 539, 762, 762, 3, 509, - 509, 888, 425, 425, 425, 425, 765, 512, 510, 844, - 800, 139, 861, 862, 512, 512, 512, 521, 515, 520, - 513, 512, 512, 494, 494, 512, 512, 883, 511, 800, - 877, 539, 800, 800, 827, 876, 512, 512, 512, 494, - 801, 801, 145, 844, 171, 132, 159, 279, 284, 432, - 443, 511, 145, 840, 800, 410, 870, 800, 858, 800, - 425, 511, 666, 800, 865, 544, 511, 511, 155, 813, - 733, 734, 758, 689, 758, 888, 799, 894, 894, 249, - 511, 734, 470, 942, 40, 59, 551, 561, 568, 866, - 515, 732, 500, 496, 694, 692, 291, 835, 838, 694, - 4, 888, 710, 290, 448, 707, 515, 243, 883, 674, - 59, 873, 511, 540, 59, 265, 425, 800, 274, 631, - 511, 152, 511, 609, 202, 628, 629, 590, 40, 175, - 599, 625, 590, 26, 137, 353, 355, 383, 527, 528, - 529, 535, 536, 152, 641, 152, 641, 600, 614, 600, - 512, 515, 541, 593, 500, 513, 512, 515, 425, 369, - 93, 425, 547, 369, 425, 425, 425, 369, 948, 521, - 510, 521, 688, 688, 688, 847, 281, 281, 512, 511, - 648, 400, 401, 541, 663, 623, 654, 538, 574, 511, - 40, 624, 645, 901, 351, 413, 541, 570, 571, 628, - 538, 538, 962, 538, 512, 515, 287, 604, 287, 289, - 603, 885, 473, 961, 538, 604, 40, 538, 512, 413, - 800, 152, 538, 892, 651, 662, 923, 657, 541, 541, - 279, 628, 504, 628, 541, 504, 628, 541, 512, 512, - 856, 171, 132, 284, 511, 757, 754, 511, 512, 512, - 512, 539, 699, 758, 683, 683, 683, 683, 538, 538, - 538, 59, 185, 721, 751, 883, 511, 739, 740, 741, - 803, 886, 883, 166, 80, 761, 760, 512, 512, 509, - 765, 512, 515, 512, 888, 510, 888, 512, 788, 790, - 791, 792, 791, 792, 792, 512, 421, 800, 143, 800, - 830, 840, 786, 786, 512, 800, 877, 878, 879, 40, - 198, 512, 880, 799, 800, 36, 36, 800, 512, 800, - 171, 511, 848, 800, 512, 145, 801, 801, 145, 145, - 800, 800, 510, 521, 511, 867, 690, 470, 800, 300, - 817, 515, 712, 688, 712, 512, 899, 800, 357, 559, - 539, 265, 319, 117, 303, 511, 549, 665, 512, 515, - 555, 942, 800, 163, 230, 511, 694, 290, 538, 512, - 884, 180, 666, 667, 873, 884, 885, 885, 512, 152, - 629, 616, 629, 590, 618, 515, 512, 119, 206, 272, - 274, 615, 511, 33, 59, 636, 625, 74, 80, 93, - 117, 119, 206, 274, 279, 327, 343, 442, 451, 595, - 596, 610, 175, 117, 190, 274, 604, 581, 107, 117, - 175, 274, 399, 402, 583, 604, 383, 529, 436, 885, - 539, 533, 3, 37, 42, 49, 55, 81, 83, 89, - 101, 170, 172, 175, 176, 193, 208, 221, 222, 224, - 234, 236, 246, 268, 273, 277, 291, 299, 301, 320, - 360, 377, 384, 388, 406, 415, 435, 441, 461, 502, - 503, 541, 582, 591, 630, 765, 838, 889, 963, 969, - 545, 627, 885, 885, 885, 885, 885, 885, 885, 885, - 885, 885, 655, 865, 865, 512, 512, 512, 689, 107, - 369, 581, 663, 511, 622, 665, 908, 40, 635, 191, - 538, 512, 515, 512, 574, 511, 40, 613, 611, 619, - 86, 578, 107, 272, 624, 665, 645, 665, 618, 451, - 905, 641, 512, 515, 628, 801, 171, 511, 848, 759, - 512, 515, 512, 712, 538, 538, 538, 538, 30, 103, - 181, 363, 511, 713, 714, 715, 716, 717, 718, 719, - 800, 800, 472, 814, 512, 802, 841, 842, 198, 180, - 736, 740, 511, 512, 742, 743, 744, 892, 767, 888, - 767, 539, 767, 510, 510, 800, 515, 512, 539, 800, - 802, 800, 800, 800, 848, 512, 800, 36, 36, 800, - 800, 145, 512, 503, 865, 512, 846, 512, 800, 512, - 511, 539, 818, 721, 512, 721, 541, 512, 872, 457, - 412, 450, 560, 539, 554, 564, 289, 557, 500, 568, - 559, 840, 59, 512, 512, 456, 457, 671, 590, 616, - 512, 512, 473, 621, 120, 194, 204, 119, 453, 800, - 117, 40, 511, 892, 885, 801, 120, 194, 119, 279, - 226, 538, 621, 88, 636, 191, 279, 582, 800, 636, - 279, 502, 503, 585, 539, 765, 641, 641, 3, 246, - 406, 889, 893, 500, 425, 425, 510, 510, 688, 512, - 512, 655, 451, 650, 652, 628, 512, 961, 40, 413, - 800, 413, 274, 511, 541, 908, 622, 151, 665, 149, - 200, 603, 122, 137, 326, 961, 107, 908, 473, 959, - 40, 289, 539, 904, 511, 662, 801, 848, 512, 512, - 9, 350, 705, 721, 511, 385, 511, 512, 515, 539, - 815, 816, 334, 722, 515, 512, 511, 540, 59, 512, - 846, 198, 512, 743, 510, 765, 877, 510, 191, 512, - 800, 800, 800, 521, 510, 521, 512, 512, 539, 819, - 814, 541, 814, 515, 456, 866, 512, 515, 91, 559, - 800, 512, 884, 884, 343, 621, 511, 612, 590, 512, - 190, 511, 800, 274, 596, 621, 624, 885, 40, 152, - 761, 893, 506, 591, 885, 885, 512, 581, 124, 512, - 611, 665, 665, 538, 152, 40, 512, 885, 961, 30, - 85, 94, 118, 190, 203, 399, 402, 607, 607, 365, - 365, 40, 64, 74, 240, 413, 800, 538, 511, 539, - 558, 567, 808, 512, 512, 511, 814, 844, 511, 844, - 715, 40, 515, 800, 451, 700, 802, 873, 883, 747, - 689, 511, 747, 800, 865, 865, 309, 820, 722, 722, - 665, 303, 665, 554, 289, 511, 552, 538, 590, 545, - 617, 620, 403, 465, 597, 598, 511, 592, 800, 512, - 248, 633, 190, 451, 532, 506, 436, 655, 541, 908, - 603, 959, 511, 538, 665, 611, 578, 665, 74, 292, - 74, 665, 905, 800, 80, 562, 512, 515, 562, 9, - 722, 512, 714, 512, 818, 816, 367, 512, 688, 873, - 510, 510, 510, 59, 689, 700, 700, 560, 93, 567, - 133, 624, 500, 512, 515, 580, 512, 272, 605, 172, - 308, 389, 289, 601, 602, 626, 592, 800, 436, 40, - 511, 959, 603, 961, 959, 292, 292, 511, 512, 892, - 563, 892, 908, 558, 563, 512, 700, 512, 702, 512, - 512, 843, 183, 336, 365, 821, 456, 885, 512, 275, - 448, 633, 591, 620, 512, 598, 204, 122, 448, 289, - 626, 289, 601, 665, 567, 562, 693, 758, 693, 53, - 104, 438, 800, 822, 823, 822, 822, 512, 665, 758, - 383, 602, 63, 272, 356, 383, 594, 594, 959, 512, - 563, 694, 694, 823, 364, 165, 322, 165, 322, 148, - 824, 824, 824, 566, 590, 25, 117, 279, 908, 693, - 36, 104, 180, 272, 422, 758, 758, 694, 823, 364, - 297 + 0, 22, 32, 34, 35, 47, 52, 61, 75, 84, + 86, 98, 100, 111, 126, 127, 128, 137, 143, 151, + 153, 154, 167, 171, 197, 213, 239, 314, 315, 316, + 322, 325, 354, 363, 379, 386, 390, 400, 412, 416, + 436, 449, 452, 455, 460, 476, 500, 514, 526, 527, + 528, 529, 530, 537, 548, 549, 550, 553, 554, 556, + 567, 585, 633, 643, 646, 649, 653, 655, 658, 659, + 663, 669, 671, 675, 682, 686, 687, 688, 695, 700, + 716, 719, 720, 723, 725, 726, 727, 728, 730, 732, + 733, 737, 790, 791, 957, 959, 960, 961, 966, 969, + 975, 976, 977, 981, 430, 479, 958, 202, 370, 381, + 416, 466, 108, 191, 551, 958, 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, 63, 64, 67, + 68, 69, 70, 71, 72, 73, 75, 76, 77, 78, + 79, 81, 83, 84, 85, 86, 87, 88, 89, 90, + 91, 92, 94, 95, 96, 97, 98, 99, 101, 102, + 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, + 113, 114, 115, 116, 118, 120, 121, 122, 123, 124, + 125, 128, 129, 130, 131, 134, 135, 136, 137, 138, + 140, 141, 142, 144, 145, 146, 148, 149, 150, 151, + 152, 153, 154, 155, 156, 157, 158, 160, 162, 163, + 164, 165, 167, 169, 170, 172, 173, 174, 175, 176, + 177, 179, 181, 182, 183, 184, 186, 187, 188, 189, + 190, 191, 192, 193, 194, 195, 196, 197, 199, 200, + 201, 202, 203, 204, 205, 207, 208, 209, 210, 211, + 212, 213, 214, 215, 216, 218, 220, 221, 222, 223, + 224, 225, 226, 227, 228, 229, 230, 233, 234, 235, + 236, 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, 275, 276, 277, + 278, 280, 281, 282, 283, 284, 285, 287, 288, 291, + 292, 293, 296, 297, 298, 299, 300, 301, 302, 303, + 304, 305, 306, 307, 308, 309, 310, 311, 312, 313, + 318, 319, 320, 321, 322, 323, 324, 325, 326, 327, + 329, 330, 331, 332, 333, 334, 336, 337, 338, 339, + 340, 341, 342, 343, 345, 346, 347, 348, 349, 350, + 351, 352, 353, 354, 355, 356, 357, 359, 360, 361, + 362, 363, 364, 365, 366, 367, 368, 369, 370, 371, + 372, 373, 374, 375, 376, 377, 378, 381, 382, 383, + 384, 385, 386, 387, 388, 389, 391, 392, 393, 394, + 395, 397, 398, 399, 400, 401, 402, 403, 404, 405, + 406, 407, 408, 409, 410, 411, 414, 415, 417, 418, + 419, 420, 421, 422, 423, 425, 426, 427, 430, 431, + 432, 433, 434, 436, 437, 438, 439, 440, 441, 442, + 443, 446, 447, 448, 450, 451, 452, 453, 455, 456, + 457, 458, 459, 460, 461, 463, 464, 465, 466, 467, + 468, 469, 470, 471, 474, 477, 478, 479, 480, 481, + 482, 483, 484, 485, 486, 487, 488, 489, 490, 491, + 492, 493, 494, 495, 496, 497, 560, 636, 637, 638, + 866, 946, 950, 3, 175, 246, 409, 555, 560, 636, + 289, 958, 55, 171, 514, 580, 177, 240, 294, 313, + 370, 420, 422, 439, 445, 448, 631, 651, 694, 5, + 30, 325, 560, 561, 945, 3, 30, 34, 35, 36, + 37, 38, 39, 40, 41, 42, 45, 49, 53, 54, + 55, 56, 57, 58, 65, 66, 71, 72, 74, 79, + 80, 81, 82, 83, 89, 93, 100, 101, 108, 112, + 115, 117, 119, 126, 127, 132, 133, 139, 143, 147, + 152, 158, 159, 161, 164, 166, 168, 170, 171, 172, + 175, 176, 178, 180, 181, 182, 185, 193, 198, 206, + 208, 209, 215, 216, 217, 218, 219, 221, 222, 224, + 231, 232, 234, 236, 237, 246, 267, 268, 269, 273, + 274, 277, 279, 280, 282, 286, 289, 290, 294, 295, + 298, 299, 301, 302, 314, 315, 316, 317, 320, 321, + 324, 328, 335, 339, 344, 358, 361, 365, 379, 380, + 387, 390, 391, 394, 396, 409, 411, 412, 413, 416, + 418, 424, 426, 427, 428, 429, 432, 434, 435, 438, + 444, 445, 449, 454, 460, 461, 462, 464, 472, 473, + 475, 476, 483, 484, 485, 486, 487, 488, 489, 490, + 491, 492, 493, 566, 636, 640, 642, 24, 81, 97, + 146, 156, 168, 173, 202, 245, 249, 319, 334, 367, + 370, 381, 384, 402, 416, 422, 423, 433, 439, 466, + 651, 664, 665, 668, 958, 945, 100, 137, 464, 514, + 529, 530, 537, 553, 554, 567, 585, 633, 643, 646, + 653, 655, 658, 659, 663, 671, 678, 682, 688, 695, + 716, 719, 720, 723, 725, 957, 959, 961, 966, 969, + 975, 981, 108, 75, 213, 66, 79, 81, 158, 231, + 280, 290, 302, 320, 366, 411, 432, 434, 438, 460, + 514, 559, 560, 561, 726, 791, 793, 795, 805, 812, + 813, 866, 868, 869, 108, 5, 560, 562, 717, 717, + 560, 945, 30, 177, 240, 385, 426, 430, 560, 967, + 968, 979, 958, 30, 132, 746, 747, 177, 240, 370, + 385, 426, 970, 971, 979, 958, 560, 725, 737, 980, + 560, 812, 416, 743, 559, 172, 514, 963, 514, 342, + 738, 739, 945, 738, 726, 727, 975, 0, 517, 464, + 676, 122, 212, 451, 147, 217, 295, 444, 749, 750, + 795, 795, 726, 728, 730, 518, 30, 426, 430, 725, + 980, 191, 559, 945, 191, 559, 191, 812, 191, 559, + 274, 562, 512, 516, 563, 564, 514, 82, 108, 173, + 202, 245, 370, 381, 416, 439, 466, 558, 108, 725, + 559, 420, 422, 420, 422, 352, 191, 559, 559, 173, + 245, 342, 381, 416, 466, 644, 377, 202, 30, 945, + 191, 566, 251, 433, 107, 416, 416, 466, 374, 377, + 191, 560, 666, 952, 191, 942, 945, 191, 945, 514, + 674, 294, 422, 678, 3, 460, 636, 640, 679, 681, + 682, 684, 685, 676, 560, 562, 555, 717, 514, 514, + 166, 514, 726, 813, 514, 514, 559, 514, 514, 171, + 514, 514, 514, 514, 726, 791, 795, 805, 507, 563, + 40, 560, 806, 807, 806, 379, 518, 729, 37, 42, + 101, 172, 208, 224, 234, 268, 314, 321, 361, 380, + 449, 809, 807, 40, 560, 806, 808, 500, 817, 562, + 171, 503, 514, 514, 721, 968, 968, 968, 497, 223, + 516, 289, 4, 6, 7, 8, 9, 10, 39, 54, + 56, 57, 65, 71, 72, 83, 112, 115, 117, 136, + 152, 159, 164, 181, 182, 215, 216, 218, 246, 267, + 269, 274, 279, 282, 291, 339, 365, 394, 426, 427, + 435, 461, 498, 505, 506, 507, 512, 514, 519, 520, + 521, 522, 560, 562, 726, 780, 829, 832, 835, 836, + 837, 839, 840, 841, 842, 844, 845, 860, 862, 863, + 864, 865, 866, 867, 869, 870, 885, 886, 897, 919, + 924, 932, 933, 934, 946, 947, 948, 931, 933, 970, + 970, 562, 970, 497, 171, 428, 503, 516, 563, 660, + 812, 3, 170, 172, 464, 682, 962, 964, 170, 965, + 860, 903, 904, 738, 518, 514, 954, 515, 515, 515, + 528, 559, 171, 294, 703, 660, 30, 132, 744, 744, + 59, 744, 161, 166, 237, 286, 755, 757, 758, 783, + 785, 786, 787, 180, 289, 454, 289, 749, 750, 514, + 417, 978, 497, 223, 152, 26, 32, 137, 293, 350, + 354, 386, 457, 542, 545, 546, 350, 152, 40, 60, + 106, 201, 250, 260, 272, 304, 350, 356, 381, 386, + 400, 531, 534, 545, 152, 350, 386, 545, 152, 350, + 386, 545, 152, 40, 552, 860, 925, 565, 566, 564, + 3, 30, 37, 42, 49, 55, 81, 83, 89, 101, + 132, 170, 172, 175, 176, 193, 208, 221, 222, 224, + 234, 236, 246, 268, 277, 299, 301, 321, 361, 380, + 391, 409, 418, 438, 462, 464, 515, 636, 641, 860, + 906, 907, 949, 955, 860, 416, 559, 560, 515, 514, + 620, 370, 631, 651, 274, 647, 40, 466, 191, 559, + 191, 559, 635, 191, 559, 191, 559, 191, 560, 650, + 89, 691, 152, 480, 90, 129, 307, 421, 191, 560, + 152, 516, 953, 63, 357, 518, 667, 152, 518, 667, + 152, 289, 672, 673, 860, 955, 352, 515, 518, 4, + 159, 289, 435, 505, 506, 533, 536, 562, 677, 679, + 680, 683, 948, 678, 428, 514, 575, 579, 171, 860, + 904, 514, 3, 68, 69, 109, 110, 113, 114, 188, + 189, 252, 253, 254, 255, 256, 257, 258, 259, 262, + 263, 375, 376, 470, 471, 494, 495, 562, 847, 848, + 849, 850, 851, 852, 853, 854, 855, 856, 857, 858, + 909, 910, 807, 808, 860, 559, 860, 911, 505, 506, + 560, 861, 862, 886, 897, 913, 514, 860, 903, 914, + 860, 58, 171, 232, 429, 860, 904, 917, 860, 515, + 561, 514, 418, 763, 764, 764, 746, 747, 795, 219, + 741, 37, 224, 380, 809, 224, 299, 810, 795, 810, + 224, 809, 514, 224, 810, 148, 199, 797, 224, 764, + 514, 561, 514, 764, 296, 560, 562, 718, 860, 972, + 974, 906, 3, 37, 42, 49, 54, 55, 56, 57, + 71, 72, 81, 83, 89, 101, 112, 115, 164, 170, + 172, 176, 193, 208, 215, 216, 218, 221, 222, 224, + 234, 236, 246, 267, 268, 269, 277, 282, 299, 301, + 321, 339, 361, 365, 380, 387, 391, 394, 409, 418, + 426, 427, 438, 444, 461, 464, 636, 639, 823, 825, + 826, 828, 830, 832, 834, 836, 837, 838, 840, 841, + 844, 845, 908, 951, 40, 235, 560, 514, 512, 726, + 463, 843, 860, 923, 843, 843, 514, 514, 831, 831, + 324, 726, 514, 833, 522, 71, 72, 843, 860, 831, + 514, 514, 478, 500, 514, 846, 514, 846, 860, 860, + 860, 148, 935, 936, 860, 904, 905, 726, 860, 903, + 9, 566, 561, 887, 888, 889, 516, 563, 926, 563, + 514, 562, 514, 514, 562, 948, 3, 8, 11, 15, + 16, 17, 18, 19, 20, 21, 36, 40, 46, 53, + 80, 176, 193, 198, 221, 222, 236, 274, 277, 291, + 294, 391, 498, 501, 502, 503, 505, 506, 507, 508, + 509, 510, 895, 896, 897, 899, 929, 477, 871, 301, + 860, 518, 741, 514, 562, 741, 3, 117, 240, 533, + 562, 845, 973, 104, 974, 974, 560, 40, 560, 515, + 518, 676, 518, 515, 739, 942, 943, 40, 954, 660, + 192, 352, 219, 386, 728, 728, 30, 751, 752, 860, + 59, 728, 745, 163, 271, 771, 226, 272, 338, 389, + 451, 4, 9, 30, 766, 860, 505, 506, 767, 768, + 860, 862, 783, 784, 758, 757, 755, 756, 166, 786, + 284, 788, 59, 734, 735, 736, 798, 861, 933, 933, + 755, 783, 904, 235, 559, 74, 82, 93, 168, 191, + 328, 445, 560, 602, 612, 627, 82, 93, 724, 93, + 724, 514, 428, 514, 600, 244, 448, 600, 93, 518, + 428, 559, 3, 828, 533, 59, 535, 533, 533, 106, + 250, 260, 59, 428, 476, 500, 532, 265, 370, 532, + 534, 812, 93, 428, 724, 370, 559, 428, 370, 551, + 560, 575, 513, 524, 906, 906, 907, 518, 749, 750, + 13, 14, 221, 221, 428, 428, 560, 619, 624, 476, + 578, 559, 342, 381, 416, 466, 644, 377, 152, 100, + 585, 648, 653, 688, 981, 144, 825, 559, 274, 531, + 654, 274, 514, 620, 40, 274, 620, 274, 514, 645, + 274, 198, 652, 191, 560, 614, 692, 566, 152, 171, + 670, 666, 565, 516, 952, 942, 945, 945, 952, 515, + 518, 13, 678, 684, 4, 948, 4, 948, 562, 566, + 577, 584, 55, 102, 123, 141, 145, 167, 170, 186, + 279, 287, 309, 336, 581, 718, 40, 515, 860, 515, + 171, 518, 515, 317, 912, 515, 861, 861, 11, 15, + 16, 19, 20, 21, 198, 221, 291, 501, 502, 503, + 505, 506, 507, 508, 509, 510, 897, 861, 515, 814, + 815, 868, 166, 171, 915, 916, 518, 515, 40, 917, + 904, 917, 917, 171, 515, 40, 806, 514, 943, 4, + 9, 560, 759, 761, 762, 933, 931, 177, 240, 416, + 420, 422, 448, 559, 742, 473, 818, 795, 224, 795, + 289, 454, 811, 795, 224, 933, 795, 281, 281, 514, + 795, 561, 819, 820, 514, 561, 819, 518, 515, 518, + 516, 514, 828, 514, 514, 516, 39, 827, 514, 847, + 848, 849, 850, 851, 852, 853, 854, 855, 856, 857, + 858, 859, 515, 518, 831, 567, 659, 695, 722, 725, + 959, 975, 904, 905, 514, 472, 920, 921, 860, 905, + 948, 860, 890, 891, 892, 893, 843, 843, 8, 15, + 16, 19, 20, 21, 501, 502, 503, 505, 506, 507, + 508, 509, 510, 560, 895, 900, 515, 904, 426, 426, + 948, 948, 514, 560, 352, 940, 166, 513, 515, 518, + 524, 518, 523, 507, 564, 904, 948, 860, 859, 859, + 825, 860, 860, 860, 860, 860, 860, 860, 860, 5, + 566, 956, 426, 45, 413, 930, 952, 860, 860, 514, + 726, 918, 132, 159, 274, 279, 284, 435, 446, 860, + 279, 514, 860, 428, 53, 176, 193, 198, 236, 391, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 30, 38, 396, 894, 512, 516, 928, 180, 162, 872, + 365, 514, 886, 934, 171, 792, 906, 792, 514, 562, + 560, 559, 962, 548, 559, 860, 518, 515, 249, 274, + 740, 454, 662, 559, 697, 514, 560, 702, 712, 713, + 715, 41, 126, 753, 518, 454, 753, 265, 728, 365, + 366, 505, 506, 768, 770, 862, 389, 226, 290, 312, + 312, 518, 509, 4, 769, 948, 769, 365, 366, 770, + 559, 941, 278, 393, 789, 514, 943, 944, 518, 180, + 454, 198, 180, 219, 784, 756, 515, 350, 545, 514, + 191, 612, 945, 226, 274, 226, 454, 514, 605, 824, + 825, 945, 560, 191, 945, 191, 560, 26, 137, 386, + 541, 544, 566, 596, 610, 945, 566, 604, 623, 945, + 542, 945, 350, 386, 531, 545, 533, 952, 945, 533, + 952, 945, 533, 350, 386, 545, 945, 945, 945, 945, + 350, 386, 545, 945, 945, 562, 506, 860, 925, 749, + 749, 749, 462, 907, 192, 355, 748, 860, 860, 279, + 557, 562, 279, 557, 560, 333, 574, 515, 518, 287, + 171, 428, 569, 647, 466, 559, 559, 635, 559, 559, + 650, 559, 294, 631, 514, 726, 514, 152, 152, 236, + 560, 602, 612, 615, 618, 628, 630, 476, 478, 607, + 151, 725, 152, 476, 693, 152, 515, 906, 40, 152, + 3, 514, 274, 289, 560, 3, 667, 565, 667, 289, + 667, 672, 860, 575, 507, 514, 533, 576, 683, 865, + 515, 518, 40, 573, 562, 573, 274, 279, 336, 573, + 59, 573, 825, 515, 860, 860, 860, 915, 825, 861, + 861, 861, 861, 861, 861, 132, 274, 284, 861, 861, + 861, 861, 861, 861, 861, 861, 861, 861, 515, 518, + 40, 816, 860, 860, 916, 915, 825, 515, 515, 515, + 904, 825, 943, 515, 312, 509, 312, 366, 509, 514, + 514, 741, 420, 422, 420, 422, 559, 743, 743, 743, + 860, 180, 772, 811, 795, 860, 514, 795, 166, 514, + 561, 802, 811, 825, 515, 518, 819, 515, 972, 3, + 908, 39, 827, 560, 822, 822, 3, 512, 512, 948, + 428, 428, 428, 428, 825, 515, 513, 904, 860, 139, + 921, 922, 515, 515, 515, 524, 518, 523, 516, 515, + 515, 497, 497, 515, 515, 943, 514, 860, 937, 560, + 860, 860, 887, 936, 515, 515, 515, 497, 861, 861, + 145, 904, 171, 132, 159, 279, 284, 435, 446, 514, + 145, 900, 860, 413, 930, 860, 918, 860, 428, 514, + 726, 860, 925, 565, 514, 514, 155, 873, 793, 794, + 818, 749, 818, 948, 859, 954, 954, 249, 514, 794, + 473, 661, 40, 59, 698, 708, 715, 926, 518, 792, + 503, 499, 754, 752, 291, 895, 898, 754, 4, 948, + 770, 290, 451, 767, 518, 243, 943, 734, 59, 933, + 514, 561, 59, 265, 428, 860, 274, 627, 514, 152, + 514, 605, 202, 624, 625, 586, 40, 175, 595, 621, + 586, 26, 137, 354, 356, 386, 538, 539, 540, 546, + 547, 152, 667, 152, 667, 596, 610, 596, 515, 518, + 562, 589, 503, 516, 515, 518, 428, 370, 93, 428, + 724, 370, 428, 428, 428, 370, 552, 524, 513, 524, + 748, 748, 748, 907, 281, 281, 515, 514, 568, 3, + 403, 404, 562, 583, 619, 574, 559, 654, 514, 40, + 620, 645, 652, 647, 352, 416, 562, 656, 657, 624, + 559, 559, 635, 559, 515, 518, 287, 600, 287, 289, + 599, 945, 476, 634, 559, 600, 40, 559, 515, 416, + 860, 650, 584, 152, 559, 670, 952, 571, 582, 683, + 577, 562, 562, 279, 624, 507, 624, 562, 507, 624, + 562, 515, 515, 916, 171, 132, 284, 514, 817, 814, + 514, 515, 515, 515, 560, 759, 818, 743, 743, 743, + 743, 559, 559, 559, 59, 185, 781, 811, 943, 514, + 799, 800, 801, 863, 946, 943, 166, 80, 821, 820, + 515, 515, 512, 825, 515, 518, 515, 948, 513, 948, + 515, 848, 850, 851, 852, 851, 852, 852, 515, 424, + 860, 143, 860, 890, 900, 846, 846, 515, 860, 937, + 938, 939, 40, 198, 515, 940, 859, 860, 36, 36, + 860, 515, 860, 171, 514, 908, 860, 515, 145, 861, + 861, 145, 145, 860, 860, 513, 524, 514, 927, 750, + 473, 860, 300, 877, 518, 772, 748, 772, 515, 722, + 860, 358, 706, 560, 265, 320, 117, 303, 514, 696, + 725, 515, 518, 702, 661, 860, 163, 230, 514, 754, + 290, 559, 515, 944, 180, 726, 727, 933, 944, 945, + 945, 515, 152, 625, 612, 625, 586, 614, 518, 515, + 119, 206, 272, 274, 611, 514, 33, 59, 632, 621, + 74, 80, 93, 117, 119, 206, 274, 279, 328, 344, + 445, 454, 591, 592, 606, 175, 117, 190, 274, 600, + 532, 107, 117, 175, 274, 402, 405, 534, 600, 386, + 540, 439, 945, 560, 544, 3, 37, 42, 49, 55, + 81, 83, 89, 101, 170, 172, 175, 176, 193, 208, + 221, 222, 224, 234, 236, 246, 268, 273, 277, 291, + 299, 301, 321, 361, 380, 387, 391, 409, 418, 438, + 444, 464, 505, 506, 533, 562, 587, 626, 636, 642, + 825, 898, 949, 566, 623, 945, 945, 945, 945, 945, + 945, 945, 945, 945, 945, 575, 925, 925, 515, 515, + 515, 749, 107, 370, 516, 532, 583, 514, 618, 725, + 693, 40, 514, 631, 191, 559, 515, 518, 515, 654, + 514, 40, 609, 607, 615, 86, 674, 107, 272, 620, + 725, 645, 725, 652, 515, 614, 454, 690, 667, 515, + 518, 624, 861, 171, 514, 908, 819, 515, 518, 515, + 772, 559, 559, 559, 559, 30, 103, 181, 364, 514, + 773, 774, 775, 776, 777, 778, 779, 860, 860, 475, + 874, 515, 862, 901, 902, 198, 180, 796, 800, 514, + 515, 802, 803, 804, 952, 827, 948, 827, 560, 827, + 513, 513, 860, 518, 515, 560, 860, 862, 860, 860, + 860, 908, 515, 860, 36, 36, 860, 860, 145, 515, + 506, 925, 515, 906, 515, 860, 515, 514, 560, 878, + 781, 515, 781, 562, 515, 932, 460, 415, 453, 707, + 560, 701, 711, 289, 704, 503, 715, 706, 900, 59, + 515, 515, 459, 460, 731, 586, 612, 515, 515, 476, + 617, 120, 194, 204, 119, 456, 860, 117, 40, 514, + 952, 945, 861, 120, 194, 119, 279, 226, 559, 617, + 88, 632, 191, 279, 533, 860, 632, 279, 505, 506, + 536, 560, 825, 667, 667, 3, 246, 409, 949, 953, + 503, 428, 428, 513, 513, 748, 515, 515, 560, 575, + 454, 570, 572, 624, 515, 634, 40, 416, 860, 584, + 416, 274, 514, 562, 693, 618, 151, 725, 149, 200, + 599, 122, 137, 327, 634, 107, 693, 476, 982, 40, + 514, 289, 560, 689, 514, 582, 861, 908, 515, 515, + 9, 351, 765, 781, 514, 388, 514, 515, 518, 560, + 875, 876, 335, 782, 518, 515, 514, 561, 59, 515, + 906, 198, 515, 803, 513, 825, 937, 513, 191, 515, + 860, 860, 860, 524, 513, 524, 515, 515, 560, 879, + 874, 562, 874, 518, 459, 926, 515, 518, 91, 706, + 860, 515, 944, 944, 344, 617, 514, 608, 586, 515, + 190, 514, 860, 274, 592, 617, 620, 945, 40, 152, + 821, 953, 509, 587, 945, 945, 515, 532, 124, 515, + 607, 725, 725, 515, 559, 152, 40, 515, 945, 634, + 30, 85, 94, 118, 190, 203, 402, 405, 603, 603, + 366, 366, 40, 64, 74, 240, 416, 860, 584, 559, + 514, 560, 705, 714, 868, 515, 515, 514, 874, 904, + 514, 904, 775, 40, 518, 860, 454, 760, 862, 933, + 943, 807, 749, 514, 807, 860, 925, 925, 309, 880, + 782, 782, 725, 303, 725, 701, 289, 514, 699, 559, + 586, 566, 613, 616, 406, 468, 593, 594, 514, 588, + 860, 515, 248, 629, 190, 454, 543, 509, 439, 575, + 562, 693, 599, 982, 514, 559, 725, 607, 674, 725, + 74, 292, 74, 725, 515, 690, 860, 80, 709, 515, + 518, 709, 9, 782, 515, 774, 515, 878, 876, 368, + 515, 748, 933, 513, 513, 513, 59, 749, 760, 760, + 707, 93, 714, 133, 620, 503, 515, 518, 531, 515, + 272, 601, 172, 308, 392, 289, 597, 598, 622, 588, + 860, 439, 40, 514, 982, 599, 634, 982, 292, 292, + 514, 515, 952, 710, 952, 693, 705, 710, 515, 760, + 515, 762, 515, 515, 903, 183, 337, 366, 881, 459, + 945, 515, 275, 451, 629, 587, 616, 515, 594, 204, + 122, 451, 289, 622, 289, 597, 725, 714, 709, 753, + 818, 753, 53, 104, 441, 860, 882, 883, 882, 882, + 515, 725, 818, 386, 598, 63, 272, 357, 386, 590, + 590, 982, 515, 710, 754, 754, 883, 365, 165, 323, + 165, 323, 148, 884, 884, 884, 713, 586, 25, 117, + 279, 693, 753, 36, 104, 180, 272, 425, 818, 818, + 754, 883, 365, 297 +}; + +/* YYR1[RULE-NUM] -- Symbol kind of the left-hand side of rule RULE-NUM. */ +static const yytype_int16 yyr1[] = +{ + 0, 525, 526, 527, 527, 528, 528, 528, 528, 528, + 528, 528, 528, 528, 528, 528, 528, 528, 528, 528, + 528, 528, 528, 528, 528, 528, 528, 528, 528, 528, + 528, 528, 528, 528, 528, 528, 528, 528, 528, 528, + 528, 528, 528, 528, 528, 528, 537, 537, 537, 537, + 537, 537, 537, 537, 538, 538, 539, 539, 540, 540, + 540, 540, 541, 541, 542, 542, 542, 542, 542, 542, + 542, 542, 542, 542, 542, 542, 542, 542, 542, 542, + 542, 542, 542, 542, 542, 542, 542, 542, 542, 542, + 542, 542, 542, 543, 543, 544, 544, 544, 544, 545, + 545, 546, 547, 547, 547, 658, 658, 658, 658, 559, + 559, 560, 560, 560, 561, 561, 562, 563, 563, 564, + 565, 566, 566, 566, 566, 723, 723, 723, 723, 723, + 723, 723, 723, 723, 723, 723, 723, 723, 724, 724, + 695, 696, 696, 696, 696, 696, 697, 697, 698, 698, + 698, 699, 699, 699, 700, 700, 701, 702, 702, 703, + 703, 703, 704, 704, 704, 705, 705, 705, 706, 706, + 707, 707, 708, 708, 709, 709, 710, 710, 711, 711, + 712, 712, 713, 713, 714, 714, 715, 655, 655, 655, + 656, 656, 657, 657, 719, 719, 719, 653, 653, 653, + 654, 654, 649, 649, 649, 650, 650, 651, 651, 651, + 652, 652, 671, 671, 671, 672, 672, 673, 673, 674, + 674, 530, 530, 531, 531, 532, 532, 532, 533, 533, + 533, 533, 534, 534, 534, 534, 534, 534, 534, 534, + 534, 534, 534, 534, 534, 534, 535, 535, 536, 536, + 536, 669, 669, 670, 670, 957, 957, 957, 957, 957, + 957, 958, 958, 958, 960, 585, 585, 585, 586, 586, + 587, 587, 587, 587, 587, 587, 588, 588, 589, 590, + 590, 590, 590, 590, 591, 591, 591, 591, 592, 592, + 592, 592, 592, 592, 592, 592, 593, 593, 594, 594, + 595, 595, 595, 596, 597, 598, 598, 598, 598, 598, + 599, 599, 599, 599, 600, 601, 601, 602, 602, 603, + 603, 603, 603, 603, 603, 603, 603, 604, 604, 605, + 606, 606, 606, 606, 607, 607, 607, 607, 608, 609, + 609, 609, 610, 611, 611, 611, 611, 611, 611, 612, + 612, 613, 613, 614, 615, 615, 615, 616, 616, 617, + 617, 618, 618, 618, 619, 620, 620, 621, 621, 622, + 623, 623, 623, 623, 624, 624, 625, 625, 626, 626, + 626, 627, 627, 627, 627, 627, 627, 628, 628, 629, + 629, 629, 629, 630, 631, 631, 631, 631, 631, 631, + 631, 631, 632, 632, 663, 663, 663, 663, 663, 663, + 664, 664, 664, 664, 664, 664, 664, 664, 664, 664, + 664, 664, 664, 664, 664, 664, 664, 664, 665, 665, + 665, 665, 665, 665, 666, 666, 667, 667, 667, 668, + 668, 668, 643, 643, 643, 643, 643, 643, 644, 644, + 645, 645, 959, 567, 567, 567, 568, 568, 568, 569, + 569, 570, 570, 571, 571, 572, 572, 573, 573, 574, + 574, 575, 575, 576, 576, 576, 576, 576, 576, 577, + 578, 578, 579, 579, 580, 580, 581, 581, 581, 581, + 581, 581, 581, 581, 581, 581, 581, 581, 581, 581, + 581, 581, 582, 583, 583, 583, 583, 583, 584, 584, + 725, 725, 726, 726, 726, 727, 727, 727, 727, 727, + 727, 727, 727, 728, 728, 729, 729, 730, 730, 730, + 730, 730, 730, 730, 730, 730, 730, 730, 730, 730, + 730, 730, 730, 730, 730, 730, 731, 731, 732, 732, + 733, 733, 734, 734, 734, 735, 735, 736, 736, 737, + 737, 737, 738, 738, 739, 740, 740, 740, 741, 741, + 742, 742, 742, 742, 742, 742, 742, 742, 742, 743, + 743, 744, 744, 744, 745, 746, 746, 747, 747, 748, + 748, 748, 749, 749, 750, 750, 751, 751, 752, 752, + 753, 753, 753, 754, 754, 754, 755, 755, 755, 755, + 756, 756, 757, 757, 757, 757, 758, 758, 759, 759, + 759, 759, 759, 759, 760, 760, 761, 761, 762, 762, + 762, 762, 763, 764, 764, 765, 765, 766, 766, 766, + 766, 766, 767, 768, 768, 768, 769, 769, 770, 770, + 771, 771, 772, 772, 772, 773, 773, 774, 774, 775, + 775, 775, 775, 775, 776, 777, 778, 779, 780, 780, + 781, 781, 782, 782, 783, 783, 784, 784, 785, 785, + 786, 787, 787, 787, 787, 788, 788, 789, 789, 789, + 790, 790, 791, 791, 792, 792, 793, 793, 794, 794, + 795, 795, 795, 795, 795, 795, 795, 795, 795, 795, + 796, 796, 797, 797, 797, 798, 798, 799, 799, 800, + 800, 801, 801, 802, 802, 803, 804, 804, 805, 805, + 805, 805, 805, 805, 805, 805, 805, 805, 805, 806, + 806, 806, 806, 807, 807, 808, 808, 808, 808, 808, + 809, 809, 809, 809, 809, 809, 810, 810, 811, 811, + 812, 812, 812, 812, 813, 813, 814, 815, 815, 816, + 816, 817, 817, 818, 818, 819, 819, 820, 821, 821, + 822, 822, 823, 823, 824, 824, 825, 825, 825, 825, + 825, 825, 825, 825, 825, 825, 826, 826, 827, 827, + 827, 828, 828, 828, 828, 828, 828, 828, 829, 829, + 829, 829, 830, 831, 831, 832, 832, 832, 832, 832, + 832, 832, 832, 832, 832, 832, 833, 833, 834, 834, + 835, 835, 836, 837, 838, 838, 839, 839, 840, 841, + 842, 842, 842, 842, 842, 842, 843, 843, 844, 844, + 844, 844, 845, 846, 846, 846, 847, 847, 848, 848, + 849, 849, 850, 850, 851, 851, 852, 852, 853, 853, + 854, 854, 855, 855, 856, 856, 857, 857, 858, 858, + 859, 859, 859, 859, 859, 859, 859, 859, 859, 859, + 859, 859, 859, 859, 859, 859, 859, 859, 859, 859, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 860, 860, 860, 860, 860, 860, 860, 860, 860, 860, + 861, 861, 861, 861, 861, 861, 861, 861, 861, 861, + 861, 861, 861, 861, 861, 861, 861, 861, 861, 861, + 861, 861, 861, 861, 861, 862, 862, 862, 863, 863, + 863, 863, 863, 863, 863, 863, 863, 863, 863, 863, + 863, 864, 864, 864, 864, 864, 864, 865, 866, 866, + 866, 866, 866, 866, 867, 867, 868, 868, 869, 869, + 869, 869, 869, 869, 869, 869, 869, 869, 869, 869, + 869, 869, 870, 870, 871, 871, 872, 872, 872, 873, + 873, 874, 874, 875, 875, 876, 877, 877, 877, 878, + 879, 879, 880, 880, 881, 881, 881, 881, 882, 882, + 883, 883, 883, 883, 883, 884, 884, 884, 884, 884, + 885, 885, 886, 886, 887, 888, 888, 889, 889, 890, + 891, 891, 892, 892, 893, 893, 894, 894, 894, 895, + 895, 896, 896, 896, 896, 896, 896, 896, 896, 896, + 896, 896, 896, 896, 896, 897, 897, 898, 898, 899, + 899, 899, 899, 899, 899, 899, 899, 900, 900, 901, + 901, 902, 902, 903, 903, 904, 904, 905, 905, 906, + 906, 907, 907, 907, 908, 908, 909, 909, 910, 910, + 910, 910, 910, 910, 910, 910, 910, 910, 910, 910, + 910, 910, 911, 911, 912, 913, 913, 914, 914, 914, + 914, 914, 914, 915, 916, 917, 917, 917, 918, 918, + 919, 920, 920, 921, 922, 922, 923, 923, 924, 924, + 564, 564, 564, 564, 925, 925, 926, 926, 927, 927, + 927, 928, 928, 928, 928, 928, 929, 929, 930, 930, + 931, 931, 932, 932, 933, 933, 934, 934, 934, 935, + 935, 936, 936, 937, 938, 938, 939, 939, 940, 940, + 940, 941, 941, 942, 942, 943, 943, 944, 944, 945, + 946, 946, 947, 947, 947, 947, 947, 947, 947, 947, + 947, 947, 947, 947, 947, 947, 948, 949, 949, 949, + 950, 950, 950, 951, 951, 951, 952, 952, 953, 953, + 954, 954, 955, 956, 956, 720, 721, 721, 722, 722, + 722, 722, 722, 722, 646, 646, 646, 647, 647, 648, + 648, 648, 648, 688, 688, 689, 690, 690, 691, 691, + 692, 692, 693, 693, 694, 694, 529, 529, 529, 529, + 529, 529, 554, 554, 555, 555, 556, 556, 557, 557, + 558, 558, 558, 558, 558, 558, 558, 558, 558, 558, + 686, 686, 687, 675, 675, 675, 675, 676, 676, 677, + 677, 677, 678, 678, 678, 678, 678, 678, 678, 678, + 678, 678, 678, 678, 678, 678, 678, 678, 678, 678, + 678, 678, 678, 678, 678, 678, 678, 678, 678, 678, + 678, 678, 679, 679, 679, 680, 680, 681, 681, 682, + 682, 683, 683, 683, 683, 684, 685, 685, 969, 969, + 969, 969, 970, 970, 970, 970, 971, 971, 972, 973, + 973, 973, 973, 973, 973, 973, 974, 974, 716, 716, + 716, 716, 716, 717, 717, 718, 718, 961, 961, 961, + 961, 961, 962, 962, 962, 962, 962, 963, 963, 964, + 964, 965, 965, 659, 659, 660, 660, 660, 661, 661, + 662, 662, 548, 548, 549, 549, 550, 550, 550, 551, + 551, 552, 552, 966, 966, 966, 966, 967, 967, 968, + 968, 968, 975, 975, 975, 975, 975, 975, 975, 975, + 976, 976, 977, 977, 978, 978, 979, 979, 980, 980, + 553, 981, 981, 981, 981, 981, 982, 982, 982, 982, + 633, 633, 633, 634, 634, 634, 635, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 636, 636, 636, + 636, 636, 636, 636, 636, 636, 636, 637, 637, 637, + 637, 637, 637, 637, 637, 637, 637, 637, 637, 637, + 637, 637, 637, 637, 637, 637, 637, 637, 637, 637, + 637, 637, 637, 637, 637, 637, 637, 637, 637, 637, + 637, 637, 637, 637, 637, 637, 637, 637, 637, 637, + 637, 637, 637, 637, 637, 637, 637, 637, 637, 637, + 637, 638, 638, 638, 638, 638, 638, 638, 638, 638, + 638, 638, 638, 638, 638, 638, 638, 638, 638, 638, + 638, 638, 638, 638, 638, 638, 638, 638, 638, 639, + 639, 639, 639, 639, 639, 639, 639, 639, 639, 639, + 639, 639, 639, 639, 639, 639, 639, 639, 639, 639, + 639, 639, 639, 639, 639, 639, 639, 639, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 640, + 640, 640, 640, 640, 640, 640, 640, 640, 640, 641, + 641, 641, 641, 641, 641, 641, 641, 641, 641, 641, + 641, 641, 641, 641, 641, 641, 641, 641, 641, 641, + 641, 641, 641, 641, 641, 641, 641, 641, 641, 641, + 641, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642, 642, 642, 642, 642, + 642, 642, 642, 642, 642, 642 +}; + +/* YYR2[RULE-NUM] -- Number of symbols on the right-hand side of rule RULE-NUM. */ +static const yytype_int8 yyr2[] = +{ + 0, 2, 1, 3, 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, 0, 4, 6, 4, 6, + 4, 6, 4, 6, 1, 2, 3, 2, 1, 3, + 2, 3, 1, 3, 2, 5, 3, 6, 4, 6, + 6, 6, 5, 5, 6, 9, 4, 5, 7, 6, + 4, 8, 4, 2, 4, 3, 6, 4, 2, 2, + 2, 2, 1, 2, 0, 1, 2, 2, 2, 1, + 3, 4, 2, 1, 0, 2, 3, 2, 3, 1, + 2, 1, 1, 1, 1, 1, 1, 1, 2, 2, + 1, 1, 1, 1, 1, 6, 6, 8, 6, 8, + 6, 8, 6, 8, 8, 10, 8, 10, 1, 0, + 9, 1, 4, 4, 7, 2, 1, 3, 2, 2, + 0, 4, 3, 0, 1, 0, 2, 3, 5, 2, + 2, 0, 8, 5, 0, 5, 5, 7, 2, 0, + 1, 1, 1, 3, 2, 0, 1, 0, 1, 3, + 1, 3, 1, 2, 1, 3, 2, 6, 8, 5, + 1, 0, 1, 3, 2, 4, 5, 5, 8, 7, + 1, 0, 8, 11, 10, 0, 1, 0, 1, 1, + 0, 2, 3, 9, 12, 1, 3, 1, 3, 3, + 0, 4, 6, 1, 2, 1, 1, 0, 1, 2, + 2, 1, 2, 2, 1, 2, 3, 2, 2, 2, + 2, 3, 3, 3, 1, 3, 1, 0, 1, 2, + 2, 5, 7, 0, 2, 2, 2, 2, 2, 2, + 2, 1, 1, 0, 2, 9, 12, 11, 0, 2, + 1, 1, 1, 1, 1, 1, 3, 0, 1, 2, + 1, 1, 2, 2, 3, 1, 1, 2, 2, 1, + 2, 3, 5, 3, 2, 5, 1, 1, 1, 0, + 5, 7, 5, 2, 3, 1, 1, 2, 2, 0, + 3, 4, 4, 0, 3, 2, 0, 3, 1, 1, + 1, 1, 1, 1, 1, 1, 1, 1, 3, 3, + 1, 2, 2, 2, 2, 2, 2, 0, 3, 3, + 3, 0, 1, 2, 1, 2, 2, 2, 2, 3, + 4, 1, 3, 1, 1, 1, 1, 3, 1, 2, + 0, 1, 2, 0, 1, 3, 0, 2, 0, 3, + 3, 1, 5, 3, 1, 3, 1, 2, 1, 4, + 5, 5, 6, 3, 7, 4, 11, 1, 3, 2, + 2, 2, 0, 3, 1, 1, 2, 2, 2, 2, + 1, 0, 1, 2, 6, 4, 6, 4, 6, 8, + 1, 1, 1, 1, 2, 1, 2, 1, 2, 1, + 1, 1, 1, 3, 3, 3, 3, 1, 2, 2, + 1, 3, 1, 1, 1, 3, 1, 1, 0, 1, + 1, 1, 8, 11, 10, 7, 10, 9, 1, 1, + 2, 3, 8, 11, 9, 7, 0, 3, 3, 1, + 1, 3, 0, 1, 3, 1, 0, 1, 0, 1, + 0, 1, 3, 1, 1, 1, 3, 1, 0, 2, + 2, 0, 2, 0, 1, 0, 1, 1, 1, 3, + 3, 1, 1, 3, 3, 3, 3, 3, 3, 4, + 3, 2, 1, 1, 1, 1, 3, 1, 1, 3, + 1, 1, 3, 3, 3, 1, 2, 4, 4, 2, + 3, 5, 5, 1, 1, 3, 0, 11, 11, 10, + 12, 1, 2, 5, 4, 4, 4, 4, 7, 5, + 4, 7, 6, 9, 9, 4, 1, 1, 1, 1, + 1, 1, 1, 5, 1, 1, 3, 1, 2, 2, + 2, 3, 1, 3, 7, 1, 2, 0, 2, 0, + 3, 3, 4, 4, 4, 4, 3, 2, 1, 1, + 0, 1, 1, 0, 2, 1, 5, 1, 0, 2, + 2, 0, 1, 0, 3, 5, 1, 3, 4, 3, + 1, 1, 0, 2, 2, 0, 2, 2, 1, 1, + 1, 0, 2, 4, 5, 4, 2, 3, 2, 2, + 2, 2, 1, 2, 3, 0, 1, 0, 5, 1, + 4, 6, 2, 1, 0, 4, 0, 1, 1, 2, + 2, 2, 1, 1, 2, 2, 1, 1, 1, 1, + 1, 1, 3, 3, 0, 1, 3, 1, 2, 1, + 1, 1, 1, 1, 2, 4, 4, 5, 1, 1, + 2, 0, 2, 0, 1, 3, 1, 0, 1, 2, + 3, 2, 4, 2, 3, 2, 0, 1, 2, 0, + 4, 5, 1, 2, 2, 0, 1, 3, 1, 2, + 3, 3, 3, 3, 3, 3, 1, 4, 9, 9, + 3, 0, 2, 2, 0, 5, 3, 1, 3, 5, + 3, 1, 2, 1, 3, 5, 1, 2, 3, 4, + 5, 4, 5, 4, 6, 5, 4, 5, 5, 5, + 2, 4, 1, 1, 0, 1, 4, 5, 4, 0, + 2, 2, 2, 1, 1, 1, 1, 0, 4, 2, + 1, 2, 2, 4, 2, 6, 2, 1, 3, 4, + 0, 2, 0, 2, 0, 1, 3, 3, 2, 0, + 2, 4, 1, 1, 1, 0, 2, 3, 5, 6, + 2, 3, 1, 5, 5, 5, 3, 3, 3, 4, + 0, 1, 1, 1, 1, 1, 2, 4, 1, 1, + 1, 1, 2, 3, 0, 1, 1, 1, 1, 1, + 2, 2, 2, 2, 2, 1, 3, 0, 1, 1, + 1, 1, 5, 2, 1, 1, 1, 1, 4, 1, + 2, 2, 1, 3, 3, 2, 1, 0, 5, 2, + 5, 2, 1, 3, 3, 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, 3, 3, 3, 3, 3, 3, 0, + 1, 3, 3, 5, 2, 2, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 3, 2, 2, 3, 3, 2, 2, 3, 3, 5, + 4, 6, 3, 5, 4, 6, 4, 6, 5, 7, + 3, 2, 4, 3, 2, 3, 3, 3, 3, 4, + 3, 4, 3, 4, 5, 6, 6, 7, 6, 7, + 6, 7, 3, 4, 4, 6, 1, 4, 3, 5, + 1, 3, 2, 2, 3, 3, 3, 3, 3, 3, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 2, + 2, 5, 6, 6, 7, 1, 1, 2, 1, 1, + 2, 2, 3, 1, 2, 4, 1, 1, 2, 2, + 4, 1, 1, 3, 1, 4, 1, 3, 3, 6, + 7, 9, 7, 7, 5, 1, 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, 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, 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, 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, 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, 6, 6, 1, 1, + 1, 1, 1, 1, 2, 1, 1, 1, 1, 1, + 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, 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, 8, 2, 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, 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, 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, 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, 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, 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, + 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, 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, 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, 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, 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 }; -#define yyerrok (yyerrstatus = 0) -#define yyclearin (yychar = YYEMPTY) -#define YYEMPTY (-2) -#define YYEOF 0 -#define YYACCEPT goto yyacceptlab -#define YYABORT goto yyabortlab -#define YYERROR goto yyerrorlab +enum { YYENOMEM = -2 }; +#define yyerrok (yyerrstatus = 0) +#define yyclearin (yychar = YYEMPTY) -/* Like YYERROR except do call yyerror. This remains here temporarily - to ease the transition to the new meaning of YYERROR, for GCC. - Once GCC version 2 has supplanted version 1, this can go. */ +#define YYACCEPT goto yyacceptlab +#define YYABORT goto yyabortlab +#define YYERROR goto yyerrorlab +#define YYNOMEM goto yyexhaustedlab -#define YYFAIL goto yyerrlab #define YYRECOVERING() (!!yyerrstatus) -#define YYBACKUP(Token, Value) \ -do \ - if (yychar == YYEMPTY && yylen == 1) \ - { \ - yychar = (Token); \ - yylval = (Value); \ - yytoken = YYTRANSLATE (yychar); \ - YYPOPSTACK (1); \ - goto yybackup; \ - } \ - else \ - { \ - yyerror (&yylloc, yyscanner, YY_("syntax error: cannot back up")); \ - YYERROR; \ - } \ -while (YYID (0)) - - -#define YYTERROR 1 -#define YYERRCODE 256 - +#define YYBACKUP(Token, Value) \ + do \ + if (yychar == YYEMPTY) \ + { \ + yychar = (Token); \ + yylval = (Value); \ + YYPOPSTACK (yylen); \ + yystate = *yyssp; \ + goto yybackup; \ + } \ + else \ + { \ + yyerror (&yylloc, yyscanner, YY_("syntax error: cannot back up")); \ + YYERROR; \ + } \ + while (0) + +/* Backward compatibility with an undocumented macro. + Use YYerror or YYUNDEF. */ +#define YYERRCODE YYUNDEF /* YYLLOC_DEFAULT -- Set CURRENT to span from RHS[1] to RHS[N]. If N is 0, then set CURRENT to the empty location which ends the previous symbol: RHS[0] (always defined). */ -#define YYRHSLOC(Rhs, K) ((Rhs)[K]) #ifndef YYLLOC_DEFAULT -# define YYLLOC_DEFAULT(Current, Rhs, N) \ - do \ - if (YYID (N)) \ - { \ - (Current).first_line = YYRHSLOC (Rhs, 1).first_line; \ - (Current).first_column = YYRHSLOC (Rhs, 1).first_column; \ - (Current).last_line = YYRHSLOC (Rhs, N).last_line; \ - (Current).last_column = YYRHSLOC (Rhs, N).last_column; \ - } \ - else \ - { \ - (Current).first_line = (Current).last_line = \ - YYRHSLOC (Rhs, 0).last_line; \ - (Current).first_column = (Current).last_column = \ - YYRHSLOC (Rhs, 0).last_column; \ - } \ - while (YYID (0)) -#endif - - -/* YY_LOCATION_PRINT -- Print the location on the stream. - This macro was not mandated originally: define only if we know - we won't break user code: when these are the locations we know. */ - -#ifndef YY_LOCATION_PRINT -# if defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL -# define YY_LOCATION_PRINT(File, Loc) \ - fprintf (File, "%d.%d-%d.%d", \ - (Loc).first_line, (Loc).first_column, \ - (Loc).last_line, (Loc).last_column) -# else -# define YY_LOCATION_PRINT(File, Loc) ((void) 0) -# endif +# define YYLLOC_DEFAULT(Current, Rhs, N) \ + do \ + if (N) \ + { \ + (Current).first_line = YYRHSLOC (Rhs, 1).first_line; \ + (Current).first_column = YYRHSLOC (Rhs, 1).first_column; \ + (Current).last_line = YYRHSLOC (Rhs, N).last_line; \ + (Current).last_column = YYRHSLOC (Rhs, N).last_column; \ + } \ + else \ + { \ + (Current).first_line = (Current).last_line = \ + YYRHSLOC (Rhs, 0).last_line; \ + (Current).first_column = (Current).last_column = \ + YYRHSLOC (Rhs, 0).last_column; \ + } \ + while (0) #endif +#define YYRHSLOC(Rhs, K) ((Rhs)[K]) -/* YYLEX -- calling `yylex' with the right arguments. */ - -#ifdef YYLEX_PARAM -# define YYLEX yylex (&yylval, &yylloc, YYLEX_PARAM) -#else -# define YYLEX yylex (&yylval, &yylloc, yyscanner) -#endif /* Enable debugging if requested. */ #if YYDEBUG @@ -18477,88 +18349,119 @@ while (YYID (0)) # define YYFPRINTF fprintf # endif -# define YYDPRINTF(Args) \ -do { \ - if (yydebug) \ - YYFPRINTF Args; \ -} while (YYID (0)) - -# define YY_SYMBOL_PRINT(Title, Type, Value, Location) \ -do { \ - if (yydebug) \ - { \ - YYFPRINTF (stderr, "%s ", Title); \ - yy_symbol_print (stderr, \ - Type, Value, Location, yyscanner); \ - YYFPRINTF (stderr, "\n"); \ - } \ -} while (YYID (0)) - - -/*--------------------------------. -| Print this symbol on YYOUTPUT. | -`--------------------------------*/ - -/*ARGSUSED*/ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static void -yy_symbol_value_print (FILE *yyoutput, int yytype, YYSTYPE const * const yyvaluep, YYLTYPE const * const yylocationp, core_yyscan_t yyscanner) -#else -static void -yy_symbol_value_print (yyoutput, yytype, yyvaluep, yylocationp, yyscanner) - FILE *yyoutput; - int yytype; - YYSTYPE const * const yyvaluep; - YYLTYPE const * const yylocationp; - core_yyscan_t yyscanner; -#endif +# define YYDPRINTF(Args) \ +do { \ + if (yydebug) \ + YYFPRINTF Args; \ +} while (0) + + +/* YYLOCATION_PRINT -- Print the location on the stream. + This macro was not mandated originally: define only if we know + we won't break user code: when these are the locations we know. */ + +# ifndef YYLOCATION_PRINT + +# if defined YY_LOCATION_PRINT + + /* Temporary convenience wrapper in case some people defined the + undocumented and private YY_LOCATION_PRINT macros. */ +# define YYLOCATION_PRINT(File, Loc) YY_LOCATION_PRINT(File, *(Loc)) + +# elif defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL + +/* Print *YYLOCP on YYO. Private, do not rely on its existence. */ + +YY_ATTRIBUTE_UNUSED +static int +yy_location_print_ (FILE *yyo, YYLTYPE const * const yylocp) { - if (!yyvaluep) - return; - YYUSE (yylocationp); - YYUSE (yyscanner); -# ifdef YYPRINT - if (yytype < YYNTOKENS) - YYPRINT (yyoutput, yytoknum[yytype], *yyvaluep); -# else - YYUSE (yyoutput); -# endif - switch (yytype) + int res = 0; + int end_col = 0 != yylocp->last_column ? yylocp->last_column - 1 : 0; + if (0 <= yylocp->first_line) { - default: - break; + res += YYFPRINTF (yyo, "%d", yylocp->first_line); + if (0 <= yylocp->first_column) + res += YYFPRINTF (yyo, ".%d", yylocp->first_column); + } + if (0 <= yylocp->last_line) + { + if (yylocp->first_line < yylocp->last_line) + { + res += YYFPRINTF (yyo, "-%d", yylocp->last_line); + if (0 <= end_col) + res += YYFPRINTF (yyo, ".%d", end_col); + } + else if (0 <= end_col && yylocp->first_column < end_col) + res += YYFPRINTF (yyo, "-%d", end_col); } + return res; } +# define YYLOCATION_PRINT yy_location_print_ + + /* Temporary convenience wrapper in case some people defined the + undocumented and private YY_LOCATION_PRINT macros. */ +# define YY_LOCATION_PRINT(File, Loc) YYLOCATION_PRINT(File, &(Loc)) + +# else + +# define YYLOCATION_PRINT(File, Loc) ((void) 0) + /* Temporary convenience wrapper in case some people defined the + undocumented and private YY_LOCATION_PRINT macros. */ +# define YY_LOCATION_PRINT YYLOCATION_PRINT + +# endif +# endif /* !defined YYLOCATION_PRINT */ + + +# define YY_SYMBOL_PRINT(Title, Kind, Value, Location) \ +do { \ + if (yydebug) \ + { \ + YYFPRINTF (stderr, "%s ", Title); \ + yy_symbol_print (stderr, \ + Kind, Value, Location, yyscanner); \ + YYFPRINTF (stderr, "\n"); \ + } \ +} while (0) -/*--------------------------------. -| Print this symbol on YYOUTPUT. | -`--------------------------------*/ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) +/*-----------------------------------. +| Print this symbol's value on YYO. | +`-----------------------------------*/ + static void -yy_symbol_print (FILE *yyoutput, int yytype, YYSTYPE const * const yyvaluep, YYLTYPE const * const yylocationp, core_yyscan_t yyscanner) -#else +yy_symbol_value_print (FILE *yyo, + yysymbol_kind_t yykind, YYSTYPE const * const yyvaluep, YYLTYPE const * const yylocationp, core_yyscan_t yyscanner) +{ + FILE *yyoutput = yyo; + YY_USE (yyoutput); + YY_USE (yylocationp); + YY_USE (yyscanner); + if (!yyvaluep) + return; + YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN + YY_USE (yykind); + YY_IGNORE_MAYBE_UNINITIALIZED_END +} + + +/*---------------------------. +| Print this symbol on YYO. | +`---------------------------*/ + static void -yy_symbol_print (yyoutput, yytype, yyvaluep, yylocationp, yyscanner) - FILE *yyoutput; - int yytype; - YYSTYPE const * const yyvaluep; - YYLTYPE const * const yylocationp; - core_yyscan_t yyscanner; -#endif +yy_symbol_print (FILE *yyo, + yysymbol_kind_t yykind, YYSTYPE const * const yyvaluep, YYLTYPE const * const yylocationp, core_yyscan_t yyscanner) { - if (yytype < YYNTOKENS) - YYFPRINTF (yyoutput, "token %s (", yytname[yytype]); - else - YYFPRINTF (yyoutput, "nterm %s (", yytname[yytype]); + YYFPRINTF (yyo, "%s %s (", + yykind < YYNTOKENS ? "token" : "nterm", yysymbol_name (yykind)); - YY_LOCATION_PRINT (yyoutput, *yylocationp); - YYFPRINTF (yyoutput, ": "); - yy_symbol_value_print (yyoutput, yytype, yyvaluep, yylocationp, yyscanner); - YYFPRINTF (yyoutput, ")"); + YYLOCATION_PRINT (yyo, yylocationp); + YYFPRINTF (yyo, ": "); + yy_symbol_value_print (yyo, yykind, yyvaluep, yylocationp, yyscanner); + YYFPRINTF (yyo, ")"); } /*------------------------------------------------------------------. @@ -18566,82 +18469,69 @@ yy_symbol_print (yyoutput, yytype, yyvaluep, yylocationp, yyscanner) | TOP (included). | `------------------------------------------------------------------*/ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static void -yy_stack_print (yytype_int16 *bottom, yytype_int16 *top) -#else static void -yy_stack_print (bottom, top) - yytype_int16 *bottom; - yytype_int16 *top; -#endif +yy_stack_print (yy_state_t *yybottom, yy_state_t *yytop) { YYFPRINTF (stderr, "Stack now"); - for (; bottom <= top; ++bottom) - YYFPRINTF (stderr, " %d", *bottom); + for (; yybottom <= yytop; yybottom++) + { + int yybot = *yybottom; + YYFPRINTF (stderr, " %d", yybot); + } YYFPRINTF (stderr, "\n"); } -# define YY_STACK_PRINT(Bottom, Top) \ -do { \ - if (yydebug) \ - yy_stack_print ((Bottom), (Top)); \ -} while (YYID (0)) +# define YY_STACK_PRINT(Bottom, Top) \ +do { \ + if (yydebug) \ + yy_stack_print ((Bottom), (Top)); \ +} while (0) /*------------------------------------------------. | Report that the YYRULE is going to be reduced. | `------------------------------------------------*/ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static void -yy_reduce_print (YYSTYPE *yyvsp, YYLTYPE *yylsp, int yyrule, core_yyscan_t yyscanner) -#else static void -yy_reduce_print (yyvsp, yylsp, yyrule, yyscanner) - YYSTYPE *yyvsp; - YYLTYPE *yylsp; - int yyrule; - core_yyscan_t yyscanner; -#endif +yy_reduce_print (yy_state_t *yyssp, YYSTYPE *yyvsp, YYLTYPE *yylsp, + int yyrule, core_yyscan_t yyscanner) { + int yylno = yyrline[yyrule]; int yynrhs = yyr2[yyrule]; int yyi; - unsigned long int yylno = yyrline[yyrule]; - YYFPRINTF (stderr, "Reducing stack by rule %d (line %lu):\n", - yyrule - 1, yylno); + YYFPRINTF (stderr, "Reducing stack by rule %d (line %d):\n", + yyrule - 1, yylno); /* The symbols being reduced. */ for (yyi = 0; yyi < yynrhs; yyi++) { - fprintf (stderr, " $%d = ", yyi + 1); - yy_symbol_print (stderr, yyrhs[yyprhs[yyrule] + yyi], - &(yyvsp[(yyi + 1) - (yynrhs)]) - , &(yylsp[(yyi + 1) - (yynrhs)]) , yyscanner); - fprintf (stderr, "\n"); + YYFPRINTF (stderr, " $%d = ", yyi + 1); + yy_symbol_print (stderr, + YY_ACCESSING_SYMBOL (+yyssp[yyi + 1 - yynrhs]), + &yyvsp[(yyi + 1) - (yynrhs)], + &(yylsp[(yyi + 1) - (yynrhs)]), yyscanner); + YYFPRINTF (stderr, "\n"); } } -# define YY_REDUCE_PRINT(Rule) \ -do { \ - if (yydebug) \ - yy_reduce_print (yyvsp, yylsp, Rule, yyscanner); \ -} while (YYID (0)) +# define YY_REDUCE_PRINT(Rule) \ +do { \ + if (yydebug) \ + yy_reduce_print (yyssp, yyvsp, yylsp, Rule, yyscanner); \ +} while (0) /* Nonzero means print parse trace. It is left uninitialized so that multiple parsers can coexist. */ int yydebug; #else /* !YYDEBUG */ -# define YYDPRINTF(Args) -# define YY_SYMBOL_PRINT(Title, Type, Value, Location) +# define YYDPRINTF(Args) ((void) 0) +# define YY_SYMBOL_PRINT(Title, Kind, Value, Location) # define YY_STACK_PRINT(Bottom, Top) # define YY_REDUCE_PRINT(Rule) #endif /* !YYDEBUG */ /* YYINITDEPTH -- initial size of the parser's stacks. */ -#ifndef YYINITDEPTH +#ifndef YYINITDEPTH # define YYINITDEPTH 200 #endif @@ -18656,270 +18546,30 @@ int yydebug; # define YYMAXDEPTH 10000 #endif - - -#if YYERROR_VERBOSE - -# ifndef yystrlen -# if defined __GLIBC__ && defined _STRING_H -# define yystrlen strlen -# else -/* Return the length of YYSTR. */ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static YYSIZE_T -yystrlen (const char *yystr) -#else -static YYSIZE_T -yystrlen (yystr) - const char *yystr; -#endif -{ - YYSIZE_T yylen; - for (yylen = 0; yystr[yylen]; yylen++) - continue; - return yylen; -} -# endif -# endif - -# ifndef yystpcpy -# if defined __GLIBC__ && defined _STRING_H && defined _GNU_SOURCE -# define yystpcpy stpcpy -# else -/* Copy YYSRC to YYDEST, returning the address of the terminating '\0' in - YYDEST. */ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static char * -yystpcpy (char *yydest, const char *yysrc) -#else -static char * -yystpcpy (yydest, yysrc) - char *yydest; - const char *yysrc; -#endif -{ - char *yyd = yydest; - const char *yys = yysrc; - - while ((*yyd++ = *yys++) != '\0') - continue; - - return yyd - 1; -} -# endif -# endif - -# ifndef yytnamerr -/* Copy to YYRES the contents of YYSTR after stripping away unnecessary - quotes and backslashes, so that it's suitable for yyerror. The - heuristic is that double-quoting is unnecessary unless the string - contains an apostrophe, a comma, or backslash (other than - backslash-backslash). YYSTR is taken from yytname. If YYRES is - null, do not copy; instead, return the length of what the result - would have been. */ -static YYSIZE_T -yytnamerr (char *yyres, const char *yystr) -{ - if (*yystr == '"') - { - YYSIZE_T yyn = 0; - char const *yyp = yystr; - - for (;;) - switch (*++yyp) - { - case '\'': - case ',': - goto do_not_strip_quotes; - - case '\\': - if (*++yyp != '\\') - goto do_not_strip_quotes; - /* Fall through. */ - default: - if (yyres) - yyres[yyn] = *yyp; - yyn++; - break; - - case '"': - if (yyres) - yyres[yyn] = '\0'; - return yyn; - } - do_not_strip_quotes: ; - } - if (! yyres) - return yystrlen (yystr); - return yystpcpy (yyres, yystr) - yyres; -} -# endif -/* Copy into YYRESULT an error message about the unexpected token - YYCHAR while in state YYSTATE. Return the number of bytes copied, - including the terminating null byte. If YYRESULT is null, do not - copy anything; just return the number of bytes that would be - copied. As a special case, return 0 if an ordinary "syntax error" - message will do. Return YYSIZE_MAXIMUM if overflow occurs during - size calculation. */ -static YYSIZE_T -yysyntax_error (char *yyresult, int yystate, int yychar) -{ - int yyn = yypact[yystate]; - if (! (YYPACT_NINF < yyn && yyn <= YYLAST)) - return 0; - else - { - int yytype = YYTRANSLATE (yychar); - YYSIZE_T yysize0 = yytnamerr (0, yytname[yytype]); - YYSIZE_T yysize = yysize0; - YYSIZE_T yysize1; - int yysize_overflow = 0; - enum { YYERROR_VERBOSE_ARGS_MAXIMUM = 5 }; - char const *yyarg[YYERROR_VERBOSE_ARGS_MAXIMUM]; - int yyx; - -# if 0 - /* This is so xgettext sees the translatable formats that are - constructed on the fly. */ - YY_("syntax error, unexpected %s"); - YY_("syntax error, unexpected %s, expecting %s"); - YY_("syntax error, unexpected %s, expecting %s or %s"); - YY_("syntax error, unexpected %s, expecting %s or %s or %s"); - YY_("syntax error, unexpected %s, expecting %s or %s or %s or %s"); -# endif - char *yyfmt; - char const *yyf; - static char const yyunexpected[] = "syntax error, unexpected %s"; - static char const yyexpecting[] = ", expecting %s"; - static char const yyor[] = " or %s"; - char yyformat[sizeof yyunexpected - + sizeof yyexpecting - 1 - + ((YYERROR_VERBOSE_ARGS_MAXIMUM - 2) - * (sizeof yyor - 1))]; - char const *yyprefix = yyexpecting; - - /* Start YYX at -YYN if negative to avoid negative indexes in - YYCHECK. */ - int yyxbegin = yyn < 0 ? -yyn : 0; - - /* Stay within bounds of both yycheck and yytname. */ - int yychecklim = YYLAST - yyn + 1; - int yyxend = yychecklim < YYNTOKENS ? yychecklim : YYNTOKENS; - int yycount = 1; - - yyarg[0] = yytname[yytype]; - yyfmt = yystpcpy (yyformat, yyunexpected); - - for (yyx = yyxbegin; yyx < yyxend; ++yyx) - if (yycheck[yyx + yyn] == yyx && yyx != YYTERROR) - { - if (yycount == YYERROR_VERBOSE_ARGS_MAXIMUM) - { - yycount = 1; - yysize = yysize0; - yyformat[sizeof yyunexpected - 1] = '\0'; - break; - } - yyarg[yycount++] = yytname[yyx]; - yysize1 = yysize + yytnamerr (0, yytname[yyx]); - yysize_overflow |= (yysize1 < yysize); - yysize = yysize1; - yyfmt = yystpcpy (yyfmt, yyprefix); - yyprefix = yyor; - } - - yyf = YY_(yyformat); - yysize1 = yysize + yystrlen (yyf); - yysize_overflow |= (yysize1 < yysize); - yysize = yysize1; - - if (yysize_overflow) - return YYSIZE_MAXIMUM; - - if (yyresult) - { - /* Avoid sprintf, as that infringes on the user's name space. - Don't have undefined behavior even if the translation - produced a string with the wrong number of "%s"s. */ - char *yyp = yyresult; - int yyi = 0; - while ((*yyp = *yyf) != '\0') - { - if (*yyp == '%' && yyf[1] == 's' && yyi < yycount) - { - yyp += yytnamerr (yyp, yyarg[yyi++]); - yyf += 2; - } - else - { - yyp++; - yyf++; - } - } - } - return yysize; - } -} -#endif /* YYERROR_VERBOSE */ - /*-----------------------------------------------. | Release the memory associated to this symbol. | `-----------------------------------------------*/ -/*ARGSUSED*/ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -static void -yydestruct (const char *yymsg, int yytype, YYSTYPE *yyvaluep, YYLTYPE *yylocationp, core_yyscan_t yyscanner) -#else static void -yydestruct (yymsg, yytype, yyvaluep, yylocationp, yyscanner) - const char *yymsg; - int yytype; - YYSTYPE *yyvaluep; - YYLTYPE *yylocationp; - core_yyscan_t yyscanner; -#endif +yydestruct (const char *yymsg, + yysymbol_kind_t yykind, YYSTYPE *yyvaluep, YYLTYPE *yylocationp, core_yyscan_t yyscanner) { - YYUSE (yyvaluep); - YYUSE (yylocationp); - YYUSE (yyscanner); - + YY_USE (yyvaluep); + YY_USE (yylocationp); + YY_USE (yyscanner); if (!yymsg) yymsg = "Deleting"; - YY_SYMBOL_PRINT (yymsg, yytype, yyvaluep, yylocationp); - - switch (yytype) - { + YY_SYMBOL_PRINT (yymsg, yykind, yyvaluep, yylocationp); - default: - break; - } + YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN + YY_USE (yykind); + YY_IGNORE_MAYBE_UNINITIALIZED_END } - - -/* Prevent warnings from -Wmissing-prototypes. */ - -#ifdef YYPARSE_PARAM -#if defined __STDC__ || defined __cplusplus -int yyparse (void *YYPARSE_PARAM); -#else -int yyparse (); -#endif -#else /* ! YYPARSE_PARAM */ -#if defined __STDC__ || defined __cplusplus -int yyparse (core_yyscan_t yyscanner); -#else -int yyparse (); -#endif -#endif /* ! YYPARSE_PARAM */ @@ -18930,221 +18580,217 @@ int yyparse (); | yyparse. | `----------*/ -#ifdef YYPARSE_PARAM -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) -int -yyparse (void *YYPARSE_PARAM) -#else -int -yyparse (YYPARSE_PARAM) - void *YYPARSE_PARAM; -#endif -#else /* ! YYPARSE_PARAM */ -#if (defined __STDC__ || defined __C99__FUNC__ \ - || defined __cplusplus || defined _MSC_VER) int yyparse (core_yyscan_t yyscanner) -#else -int -yyparse (yyscanner) - core_yyscan_t yyscanner; -#endif -#endif { - /* The look-ahead symbol. */ +/* Lookahead token kind. */ int yychar; -/* The semantic value of the look-ahead symbol. */ -YYSTYPE yylval; -/* Number of syntax errors so far. */ -int yynerrs; -/* Location data for the look-ahead symbol. */ -YYLTYPE yylloc; +/* The semantic value of the lookahead symbol. */ +/* Default value used for initialization, for pacifying older GCCs + or non-GCC compilers. */ +YY_INITIAL_VALUE (static YYSTYPE yyval_default;) +YYSTYPE yylval YY_INITIAL_VALUE (= yyval_default); - int yystate; - int yyn; - int yyresult; - /* Number of tokens to shift before error messages enabled. */ - int yyerrstatus; - /* Look-ahead token as an internal (translated) token number. */ - int yytoken = 0; -#if YYERROR_VERBOSE - /* Buffer for error messages, and its allocated size. */ - char yymsgbuf[128]; - char *yymsg = yymsgbuf; - YYSIZE_T yymsg_alloc = sizeof yymsgbuf; -#endif +/* Location data for the lookahead symbol. */ +static YYLTYPE yyloc_default +# if defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL + = { 1, 1, 1, 1 } +# endif +; +YYLTYPE yylloc = yyloc_default; - /* Three stacks and their tools: - `yyss': related to states, - `yyvs': related to semantic values, - `yyls': related to locations. + /* Number of syntax errors so far. */ + int yynerrs = 0; (void)yynerrs; - Refer to the stacks thru separate pointers, to allow yyoverflow - to reallocate them elsewhere. */ + yy_state_fast_t yystate = 0; + /* Number of tokens to shift before error messages enabled. */ + int yyerrstatus = 0; - /* The state stack. */ - yytype_int16 yyssa[YYINITDEPTH]; - yytype_int16 *yyss = yyssa; - yytype_int16 *yyssp; + /* Refer to the stacks through separate pointers, to allow yyoverflow + to reallocate them elsewhere. */ - /* The semantic value stack. */ - YYSTYPE yyvsa[YYINITDEPTH]; - YYSTYPE *yyvs = yyvsa; - YYSTYPE *yyvsp; + /* Their size. */ + YYPTRDIFF_T yystacksize = YYINITDEPTH; - /* The location stack. */ - YYLTYPE yylsa[YYINITDEPTH]; - YYLTYPE *yyls = yylsa; - YYLTYPE *yylsp; - /* The locations where the error started and ended. */ - YYLTYPE yyerror_range[2]; + /* The state stack: array, bottom, top. */ + yy_state_t yyssa[YYINITDEPTH]; + yy_state_t *yyss = yyssa; + yy_state_t *yyssp = yyss; -#define YYPOPSTACK(N) (yyvsp -= (N), yyssp -= (N), yylsp -= (N)) + /* The semantic value stack: array, bottom, top. */ + YYSTYPE yyvsa[YYINITDEPTH]; + YYSTYPE *yyvs = yyvsa; + YYSTYPE *yyvsp = yyvs; - YYSIZE_T yystacksize = YYINITDEPTH; + /* The location stack: array, bottom, top. */ + YYLTYPE yylsa[YYINITDEPTH]; + YYLTYPE *yyls = yylsa; + YYLTYPE *yylsp = yyls; + int yyn; + /* The return value of yyparse. */ + int yyresult; + /* Lookahead symbol kind. */ + yysymbol_kind_t yytoken = YYSYMBOL_YYEMPTY; /* The variables used to return semantic value and location from the action routines. */ YYSTYPE yyval; YYLTYPE yyloc; + /* The locations where the error started and ended. */ + YYLTYPE yyerror_range[3]; + + + +#define YYPOPSTACK(N) (yyvsp -= (N), yyssp -= (N), yylsp -= (N)) + /* The number of symbols on the RHS of the reduced rule. Keep to zero when no symbol should be popped. */ int yylen = 0; YYDPRINTF ((stderr, "Starting parse\n")); - yystate = 0; - yyerrstatus = 0; - yynerrs = 0; (void)yynerrs; - yychar = YYEMPTY; /* Cause a token to be read. */ - - /* Initialize stack pointers. - Waste one element of value and location stack - so that they stay on the same level as the state stack. - The wasted elements are never initialized. */ - - yyssp = yyss; - yyvsp = yyvs; - yylsp = yyls; -#if defined YYLTYPE_IS_TRIVIAL && YYLTYPE_IS_TRIVIAL - /* Initialize the default location before parsing starts. */ - yylloc.first_line = yylloc.last_line = 1; - yylloc.first_column = yylloc.last_column = 0; -#endif + yychar = YYEMPTY; /* Cause a token to be read. */ + yylsp[0] = yylloc; goto yysetstate; + /*------------------------------------------------------------. -| yynewstate -- Push a new state, which is found in yystate. | +| yynewstate -- push a new state, which is found in yystate. | `------------------------------------------------------------*/ - yynewstate: +yynewstate: /* In all cases, when you get here, the value and location stacks have just been pushed. So pushing a state here evens the stacks. */ yyssp++; - yysetstate: - *yyssp = yystate; + +/*--------------------------------------------------------------------. +| yysetstate -- set current state (the top of the stack) to yystate. | +`--------------------------------------------------------------------*/ +yysetstate: + YYDPRINTF ((stderr, "Entering state %d\n", yystate)); + YY_ASSERT (0 <= yystate && yystate < YYNSTATES); + YY_IGNORE_USELESS_CAST_BEGIN + *yyssp = YY_CAST (yy_state_t, yystate); + YY_IGNORE_USELESS_CAST_END + YY_STACK_PRINT (yyss, yyssp); if (yyss + yystacksize - 1 <= yyssp) +#if !defined yyoverflow && !defined YYSTACK_RELOCATE + YYNOMEM; +#else { /* Get the current used size of the three stacks, in elements. */ - YYSIZE_T yysize = yyssp - yyss + 1; + YYPTRDIFF_T yysize = yyssp - yyss + 1; -#ifdef yyoverflow +# if defined yyoverflow { - /* Give user a chance to reallocate the stack. Use copies of - these so that the &'s don't force the real ones into - memory. */ - YYSTYPE *yyvs1 = yyvs; - yytype_int16 *yyss1 = yyss; - YYLTYPE *yyls1 = yyls; - - /* Each stack pointer address is followed by the size of the - data in use in that stack, in bytes. This used to be a - conditional around just the two extra args, but that might - be undefined if yyoverflow is a macro. */ - yyoverflow (YY_("memory exhausted"), - &yyss1, yysize * sizeof (*yyssp), - &yyvs1, yysize * sizeof (*yyvsp), - &yyls1, yysize * sizeof (*yylsp), - &yystacksize); - yyls = yyls1; - yyss = yyss1; - yyvs = yyvs1; + /* Give user a chance to reallocate the stack. Use copies of + these so that the &'s don't force the real ones into + memory. */ + yy_state_t *yyss1 = yyss; + YYSTYPE *yyvs1 = yyvs; + YYLTYPE *yyls1 = yyls; + + /* Each stack pointer address is followed by the size of the + data in use in that stack, in bytes. This used to be a + conditional around just the two extra args, but that might + be undefined if yyoverflow is a macro. */ + yyoverflow (YY_("memory exhausted"), + &yyss1, yysize * YYSIZEOF (*yyssp), + &yyvs1, yysize * YYSIZEOF (*yyvsp), + &yyls1, yysize * YYSIZEOF (*yylsp), + &yystacksize); + yyss = yyss1; + yyvs = yyvs1; + yyls = yyls1; } -#else /* no yyoverflow */ -# ifndef YYSTACK_RELOCATE - goto yyexhaustedlab; -# else +# else /* defined YYSTACK_RELOCATE */ /* Extend the stack our own way. */ if (YYMAXDEPTH <= yystacksize) - goto yyexhaustedlab; + YYNOMEM; yystacksize *= 2; if (YYMAXDEPTH < yystacksize) - yystacksize = YYMAXDEPTH; + yystacksize = YYMAXDEPTH; { - yytype_int16 *yyss1 = yyss; - union yyalloc *yyptr = - (union yyalloc *) YYSTACK_ALLOC (YYSTACK_BYTES (yystacksize)); - if (! yyptr) - goto yyexhaustedlab; - YYSTACK_RELOCATE (yyss); - YYSTACK_RELOCATE (yyvs); - YYSTACK_RELOCATE (yyls); + yy_state_t *yyss1 = yyss; + union yyalloc *yyptr = + YY_CAST (union yyalloc *, + YYSTACK_ALLOC (YY_CAST (YYSIZE_T, YYSTACK_BYTES (yystacksize)))); + if (! yyptr) + YYNOMEM; + YYSTACK_RELOCATE (yyss_alloc, yyss); + YYSTACK_RELOCATE (yyvs_alloc, yyvs); + YYSTACK_RELOCATE (yyls_alloc, yyls); # undef YYSTACK_RELOCATE - if (yyss1 != yyssa) - YYSTACK_FREE (yyss1); + if (yyss1 != yyssa) + YYSTACK_FREE (yyss1); } # endif -#endif /* no yyoverflow */ yyssp = yyss + yysize - 1; yyvsp = yyvs + yysize - 1; yylsp = yyls + yysize - 1; - YYDPRINTF ((stderr, "Stack size increased to %lu\n", - (unsigned long int) yystacksize)); + YY_IGNORE_USELESS_CAST_BEGIN + YYDPRINTF ((stderr, "Stack size increased to %ld\n", + YY_CAST (long, yystacksize))); + YY_IGNORE_USELESS_CAST_END if (yyss + yystacksize - 1 <= yyssp) - YYABORT; + YYABORT; } +#endif /* !defined yyoverflow && !defined YYSTACK_RELOCATE */ - YYDPRINTF ((stderr, "Entering state %d\n", yystate)); + + if (yystate == YYFINAL) + YYACCEPT; goto yybackup; + /*-----------. | yybackup. | `-----------*/ yybackup: - /* Do appropriate processing given the current state. Read a - look-ahead token if we need one and don't already have one. */ + lookahead token if we need one and don't already have one. */ - /* First try to decide what to do without reference to look-ahead token. */ + /* First try to decide what to do without reference to lookahead token. */ yyn = yypact[yystate]; - if (yyn == YYPACT_NINF) + if (yypact_value_is_default (yyn)) goto yydefault; - /* Not known => get a look-ahead token if don't already have one. */ + /* Not known => get a lookahead token if don't already have one. */ - /* YYCHAR is either YYEMPTY or YYEOF or a valid look-ahead symbol. */ + /* YYCHAR is either empty, or end-of-input, or a valid lookahead. */ if (yychar == YYEMPTY) { - YYDPRINTF ((stderr, "Reading a token: ")); - yychar = YYLEX; + YYDPRINTF ((stderr, "Reading a token\n")); + yychar = yylex (&yylval, &yylloc, yyscanner); } if (yychar <= YYEOF) { - yychar = yytoken = YYEOF; + yychar = YYEOF; + yytoken = YYSYMBOL_YYEOF; YYDPRINTF ((stderr, "Now at end of input.\n")); } + else if (yychar == YYerror) + { + /* The scanner already issued an error message, process directly + to error recovery. But do not keep the error token as + lookahead, it is too special and may lead us to an endless + loop in error recovery. */ + yychar = YYUNDEF; + yytoken = YYSYMBOL_YYerror; + yyerror_range[1] = yylloc; + goto yyerrlab1; + } else { yytoken = YYTRANSLATE (yychar); @@ -19159,30 +18805,27 @@ YYLTYPE yylloc; yyn = yytable[yyn]; if (yyn <= 0) { - if (yyn == 0 || yyn == YYTABLE_NINF) - goto yyerrlab; + if (yytable_value_is_error (yyn)) + goto yyerrlab; yyn = -yyn; goto yyreduce; } - if (yyn == YYFINAL) - YYACCEPT; - /* Count tokens shifted since error; after three, turn off error status. */ if (yyerrstatus) yyerrstatus--; - /* Shift the look-ahead token. */ + /* Shift the lookahead token. */ YY_SYMBOL_PRINT ("Shifting", yytoken, &yylval, &yylloc); - - /* Discard the shifted token unless it is eof. */ - if (yychar != YYEOF) - yychar = YYEMPTY; - yystate = yyn; + YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN *++yyvsp = yylval; + YY_IGNORE_MAYBE_UNINITIALIZED_END *++yylsp = yylloc; + + /* Discard the shifted token. */ + yychar = YYEMPTY; goto yynewstate; @@ -19197,14 +18840,14 @@ YYLTYPE yylloc; /*-----------------------------. -| yyreduce -- Do a reduction. | +| yyreduce -- do a reduction. | `-----------------------------*/ yyreduce: /* yyn is the number of a rule to reduce with. */ yylen = yyr2[yyn]; /* If YYLEN is nonzero, implement the default value of the action: - `$$ = $1'. + '$$ = $1'. Otherwise, the following line sets YYVAL to garbage. This behavior is undocumented and Bison @@ -19213,1993 +18856,2347 @@ YYLTYPE yylloc; GCC warning that YYVAL may be used uninitialized. */ yyval = yyvsp[1-yylen]; - /* Default location. */ + /* Default location. */ YYLLOC_DEFAULT (yyloc, (yylsp - yylen), yylen); + yyerror_range[1] = yyloc; YY_REDUCE_PRINT (yyn); switch (yyn) { - case 2: -#line 495 "third_party/libpg_query/grammar/grammar.y" - { - pg_yyget_extra(yyscanner)->parsetree = (yyvsp[(1) - (1)].list); - ;} + case 2: /* stmtblock: stmtmulti */ +#line 501 "third_party/libpg_query/grammar/grammar.y" + { + pg_yyget_extra(yyscanner)->parsetree = (yyvsp[0].list); + } +#line 18871 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 3: -#line 511 "third_party/libpg_query/grammar/grammar.y" - { - if ((yyvsp[(1) - (3)].list) != NIL) + case 3: /* stmtmulti: stmtmulti ';' stmt */ +#line 517 "third_party/libpg_query/grammar/grammar.y" + { + if ((yyvsp[-2].list) != NIL) { /* update length of previous stmt */ - updateRawStmtEnd(llast_node(PGRawStmt, (yyvsp[(1) - (3)].list)), (yylsp[(2) - (3)])); + updateRawStmtEnd(llast_node(PGRawStmt, (yyvsp[-2].list)), (yylsp[-1])); } - if ((yyvsp[(3) - (3)].node) != NULL) - (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeRawStmt((yyvsp[(3) - (3)].node), (yylsp[(2) - (3)]) + 1)); + if ((yyvsp[0].node) != NULL) + (yyval.list) = lappend((yyvsp[-2].list), makeRawStmt((yyvsp[0].node), (yylsp[-1]) + 1)); else - (yyval.list) = (yyvsp[(1) - (3)].list); - ;} + (yyval.list) = (yyvsp[-2].list); + } +#line 18887 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 4: -#line 523 "third_party/libpg_query/grammar/grammar.y" - { - if ((yyvsp[(1) - (1)].node) != NULL) - (yyval.list) = list_make1(makeRawStmt((yyvsp[(1) - (1)].node), 0)); + case 4: /* stmtmulti: stmt */ +#line 529 "third_party/libpg_query/grammar/grammar.y" + { + if ((yyvsp[0].node) != NULL) + (yyval.list) = list_make1(makeRawStmt((yyvsp[0].node), 0)); else (yyval.list) = NIL; - ;} + } +#line 18898 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 42: -#line 569 "third_party/libpg_query/grammar/grammar.y" - { (yyval.node) = NULL; ;} + case 45: /* stmt: %empty */ +#line 578 "third_party/libpg_query/grammar/grammar.y" + { (yyval.node) = NULL; } +#line 18904 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 43: + case 46: /* AlterTableStmt: ALTER TABLE relation_expr alter_table_cmds */ #line 10 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(3) - (4)].range); - n->cmds = (yyvsp[(4) - (4)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_TABLE; n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 18917 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 44: + case 47: /* AlterTableStmt: ALTER TABLE IF_P EXISTS relation_expr alter_table_cmds */ #line 19 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(5) - (6)].range); - n->cmds = (yyvsp[(6) - (6)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_TABLE; n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 18930 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 45: + case 48: /* AlterTableStmt: ALTER INDEX qualified_name alter_table_cmds */ #line 28 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(3) - (4)].range); - n->cmds = (yyvsp[(4) - (4)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_INDEX; n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 18943 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 46: + case 49: /* AlterTableStmt: ALTER INDEX IF_P EXISTS qualified_name alter_table_cmds */ #line 37 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(5) - (6)].range); - n->cmds = (yyvsp[(6) - (6)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_INDEX; n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 18956 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 47: + case 50: /* AlterTableStmt: ALTER SEQUENCE qualified_name alter_table_cmds */ #line 46 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(3) - (4)].range); - n->cmds = (yyvsp[(4) - (4)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_SEQUENCE; n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 18969 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 48: + case 51: /* AlterTableStmt: ALTER SEQUENCE IF_P EXISTS qualified_name alter_table_cmds */ #line 55 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(5) - (6)].range); - n->cmds = (yyvsp[(6) - (6)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_SEQUENCE; n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 18982 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 49: + case 52: /* AlterTableStmt: ALTER VIEW qualified_name alter_table_cmds */ #line 64 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(3) - (4)].range); - n->cmds = (yyvsp[(4) - (4)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_VIEW; n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 18995 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 50: + case 53: /* AlterTableStmt: ALTER VIEW IF_P EXISTS qualified_name alter_table_cmds */ #line 73 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableStmt *n = makeNode(PGAlterTableStmt); - n->relation = (yyvsp[(5) - (6)].range); - n->cmds = (yyvsp[(6) - (6)].list); + n->relation = (yyvsp[-1].range); + n->cmds = (yyvsp[0].list); n->relkind = PG_OBJECT_VIEW; n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19008 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 51: + case 54: /* alter_identity_column_option_list: alter_identity_column_option */ #line 86 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); ;} + { (yyval.list) = list_make1((yyvsp[0].defelt)); } +#line 19014 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 52: + case 55: /* alter_identity_column_option_list: alter_identity_column_option_list alter_identity_column_option */ #line 88 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].defelt)); ;} + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].defelt)); } +#line 19020 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 53: + case 56: /* alter_column_default: SET DEFAULT a_expr */ #line 93 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.node) = (yyvsp[(3) - (3)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 19026 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 54: + case 57: /* alter_column_default: DROP DEFAULT */ #line 94 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 19032 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 55: + case 58: /* alter_identity_column_option: RESTART */ #line 100 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = makeDefElem("restart", NULL, (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("restart", NULL, (yylsp[0])); + } +#line 19040 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 56: + case 59: /* alter_identity_column_option: RESTART opt_with NumericOnly */ #line 104 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = makeDefElem("restart", (PGNode *)(yyvsp[(3) - (3)].value), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("restart", (PGNode *)(yyvsp[0].value), (yylsp[-2])); + } +#line 19048 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 57: + case 60: /* alter_identity_column_option: SET SeqOptElem */ #line 108 "third_party/libpg_query/grammar/statements/alter_table.y" - { - if (strcmp((yyvsp[(2) - (2)].defelt)->defname, "as") == 0 || - strcmp((yyvsp[(2) - (2)].defelt)->defname, "restart") == 0 || - strcmp((yyvsp[(2) - (2)].defelt)->defname, "owned_by") == 0) + { + if (strcmp((yyvsp[0].defelt)->defname, "as") == 0 || + strcmp((yyvsp[0].defelt)->defname, "restart") == 0 || + strcmp((yyvsp[0].defelt)->defname, "owned_by") == 0) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), - errmsg("sequence option \"%s\" not supported here", (yyvsp[(2) - (2)].defelt)->defname), - parser_errposition((yylsp[(2) - (2)])))); - (yyval.defelt) = (yyvsp[(2) - (2)].defelt); - ;} + errmsg("sequence option \"%s\" not supported here", (yyvsp[0].defelt)->defname), + parser_errposition((yylsp[0])))); + (yyval.defelt) = (yyvsp[0].defelt); + } +#line 19063 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 58: + case 61: /* alter_identity_column_option: SET GENERATED generated_when */ #line 119 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = makeDefElem("generated", (PGNode *) makeInteger((yyvsp[(3) - (3)].ival)), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("generated", (PGNode *) makeInteger((yyvsp[0].ival)), (yylsp[-2])); + } +#line 19071 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 59: + case 62: /* alter_generic_option_list: alter_generic_option_elem */ #line 127 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].defelt)); + } +#line 19079 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 60: + case 63: /* alter_generic_option_list: alter_generic_option_list ',' alter_generic_option_elem */ #line 131 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].defelt)); + } +#line 19087 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 61: + case 64: /* alter_table_cmd: ADD_P columnDef */ #line 140 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AddColumn; - n->def = (yyvsp[(2) - (2)].node); + n->def = (yyvsp[0].node); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19099 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 62: + case 65: /* alter_table_cmd: ADD_P IF_P NOT EXISTS columnDef */ #line 149 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AddColumn; - n->def = (yyvsp[(5) - (5)].node); + n->def = (yyvsp[0].node); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19111 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 63: + case 66: /* alter_table_cmd: ADD_P COLUMN columnDef */ #line 158 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AddColumn; - n->def = (yyvsp[(3) - (3)].node); + n->def = (yyvsp[0].node); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19123 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 64: + case 67: /* alter_table_cmd: ADD_P COLUMN IF_P NOT EXISTS columnDef */ #line 167 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AddColumn; - n->def = (yyvsp[(6) - (6)].node); + n->def = (yyvsp[0].node); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19135 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 65: + case 68: /* alter_table_cmd: ALTER opt_column ColId alter_column_default */ #line 176 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_ColumnDefault; - n->name = (yyvsp[(3) - (4)].str); - n->def = (yyvsp[(4) - (4)].node); + n->name = (yyvsp[-1].str); + n->def = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 19147 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 66: + case 69: /* alter_table_cmd: ALTER opt_column ColId DROP NOT NULL_P */ #line 185 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_DropNotNull; - n->name = (yyvsp[(3) - (6)].str); + n->name = (yyvsp[-3].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 19158 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 67: + case 70: /* alter_table_cmd: ALTER opt_column ColId SET NOT NULL_P */ #line 193 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetNotNull; - n->name = (yyvsp[(3) - (6)].str); + n->name = (yyvsp[-3].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 19169 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 68: + case 71: /* alter_table_cmd: ALTER opt_column ColId SET STATISTICS SignedIconst */ #line 201 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetStatistics; - n->name = (yyvsp[(3) - (6)].str); - n->def = (PGNode *) makeInteger((yyvsp[(6) - (6)].ival)); + n->name = (yyvsp[-3].str); + n->def = (PGNode *) makeInteger((yyvsp[0].ival)); (yyval.node) = (PGNode *)n; - ;} + } +#line 19181 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 69: + case 72: /* alter_table_cmd: ALTER opt_column ColId SET reloptions */ #line 210 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetOptions; - n->name = (yyvsp[(3) - (5)].str); - n->def = (PGNode *) (yyvsp[(5) - (5)].list); + n->name = (yyvsp[-2].str); + n->def = (PGNode *) (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19193 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 70: + case 73: /* alter_table_cmd: ALTER opt_column ColId RESET reloptions */ #line 219 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_ResetOptions; - n->name = (yyvsp[(3) - (5)].str); - n->def = (PGNode *) (yyvsp[(5) - (5)].list); + n->name = (yyvsp[-2].str); + n->def = (PGNode *) (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19205 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 71: + case 74: /* alter_table_cmd: ALTER opt_column ColId SET STORAGE ColId */ #line 228 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetStorage; - n->name = (yyvsp[(3) - (6)].str); - n->def = (PGNode *) makeString((yyvsp[(6) - (6)].str)); + n->name = (yyvsp[-3].str); + n->def = (PGNode *) makeString((yyvsp[0].str)); (yyval.node) = (PGNode *)n; - ;} + } +#line 19217 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 72: + case 75: /* alter_table_cmd: ALTER opt_column ColId ADD_P GENERATED generated_when AS IDENTITY_P OptParenthesizedSeqOptList */ #line 237 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); PGConstraint *c = makeNode(PGConstraint); c->contype = PG_CONSTR_IDENTITY; - c->generated_when = (yyvsp[(6) - (9)].ival); - c->options = (yyvsp[(9) - (9)].list); - c->location = (yylsp[(5) - (9)]); + c->generated_when = (yyvsp[-3].ival); + c->options = (yyvsp[0].list); + c->location = (yylsp[-4]); n->subtype = PG_AT_AddIdentity; - n->name = (yyvsp[(3) - (9)].str); + n->name = (yyvsp[-6].str); n->def = (PGNode *) c; (yyval.node) = (PGNode *)n; - ;} + } +#line 19237 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 73: + case 76: /* alter_table_cmd: ALTER opt_column ColId alter_identity_column_option_list */ #line 254 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetIdentity; - n->name = (yyvsp[(3) - (4)].str); - n->def = (PGNode *) (yyvsp[(4) - (4)].list); + n->name = (yyvsp[-1].str); + n->def = (PGNode *) (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19249 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 74: + case 77: /* alter_table_cmd: ALTER opt_column ColId DROP IDENTITY_P */ #line 263 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = AT_DropIdentity; - n->name = (yyvsp[(3) - (5)].str); + n->name = (yyvsp[-2].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19261 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 75: + case 78: /* alter_table_cmd: ALTER opt_column ColId DROP IDENTITY_P IF_P EXISTS */ #line 272 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = AT_DropIdentity; - n->name = (yyvsp[(3) - (7)].str); + n->name = (yyvsp[-4].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19273 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 76: + case 79: /* alter_table_cmd: DROP opt_column IF_P EXISTS ColId opt_drop_behavior */ #line 281 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_DropColumn; - n->name = (yyvsp[(5) - (6)].str); - n->behavior = (yyvsp[(6) - (6)].dbehavior); + n->name = (yyvsp[-1].str); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19286 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 77: + case 80: /* alter_table_cmd: DROP opt_column ColId opt_drop_behavior */ #line 291 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_DropColumn; - n->name = (yyvsp[(3) - (4)].str); - n->behavior = (yyvsp[(4) - (4)].dbehavior); + n->name = (yyvsp[-1].str); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19299 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 78: + case 81: /* alter_table_cmd: ALTER opt_column ColId opt_set_data TYPE_P Typename opt_collate_clause alter_using */ #line 304 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); PGColumnDef *def = makeNode(PGColumnDef); n->subtype = PG_AT_AlterColumnType; - n->name = (yyvsp[(3) - (8)].str); + n->name = (yyvsp[-5].str); n->def = (PGNode *) def; /* We only use these fields of the PGColumnDef node */ - def->typeName = (yyvsp[(6) - (8)].typnam); - def->collClause = (PGCollateClause *) (yyvsp[(7) - (8)].node); - def->raw_default = (yyvsp[(8) - (8)].node); - def->location = (yylsp[(3) - (8)]); + def->typeName = (yyvsp[-2].typnam); + def->collClause = (PGCollateClause *) (yyvsp[-1].node); + def->raw_default = (yyvsp[0].node); + def->location = (yylsp[-5]); (yyval.node) = (PGNode *)n; - ;} + } +#line 19317 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 79: + case 82: /* alter_table_cmd: ALTER opt_column ColId alter_generic_options */ #line 319 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AlterColumnGenericOptions; - n->name = (yyvsp[(3) - (4)].str); - n->def = (PGNode *) (yyvsp[(4) - (4)].list); + n->name = (yyvsp[-1].str); + n->def = (PGNode *) (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19329 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 80: + case 83: /* alter_table_cmd: ADD_P TableConstraint */ #line 328 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_AddConstraint; - n->def = (yyvsp[(2) - (2)].node); + n->def = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 19340 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 81: + case 84: /* alter_table_cmd: ALTER CONSTRAINT name ConstraintAttributeSpec */ #line 336 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); PGConstraint *c = makeNode(PGConstraint); n->subtype = PG_AT_AlterConstraint; n->def = (PGNode *) c; c->contype = PG_CONSTR_FOREIGN; /* others not supported, yet */ - c->conname = (yyvsp[(3) - (4)].str); - processCASbits((yyvsp[(4) - (4)].ival), (yylsp[(4) - (4)]), "ALTER CONSTRAINT statement", + c->conname = (yyvsp[-1].str); + processCASbits((yyvsp[0].ival), (yylsp[0]), "ALTER CONSTRAINT statement", &c->deferrable, &c->initdeferred, NULL, NULL, yyscanner); (yyval.node) = (PGNode *)n; - ;} + } +#line 19358 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 82: + case 85: /* alter_table_cmd: VALIDATE CONSTRAINT name */ #line 351 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_ValidateConstraint; - n->name = (yyvsp[(3) - (3)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 19369 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 83: + case 86: /* alter_table_cmd: DROP CONSTRAINT IF_P EXISTS name opt_drop_behavior */ #line 359 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_DropConstraint; - n->name = (yyvsp[(5) - (6)].str); - n->behavior = (yyvsp[(6) - (6)].dbehavior); + n->name = (yyvsp[-1].str); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19382 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 84: + case 87: /* alter_table_cmd: DROP CONSTRAINT name opt_drop_behavior */ #line 369 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_DropConstraint; - n->name = (yyvsp[(3) - (4)].str); - n->behavior = (yyvsp[(4) - (4)].dbehavior); + n->name = (yyvsp[-1].str); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19395 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 85: + case 88: /* alter_table_cmd: SET LOGGED */ #line 379 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetLogged; (yyval.node) = (PGNode *)n; - ;} + } +#line 19405 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 86: + case 89: /* alter_table_cmd: SET UNLOGGED */ #line 386 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetUnLogged; (yyval.node) = (PGNode *)n; - ;} + } +#line 19415 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 87: + case 90: /* alter_table_cmd: SET reloptions */ #line 393 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_SetRelOptions; - n->def = (PGNode *)(yyvsp[(2) - (2)].list); + n->def = (PGNode *)(yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19426 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 88: + case 91: /* alter_table_cmd: RESET reloptions */ #line 401 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_ResetRelOptions; - n->def = (PGNode *)(yyvsp[(2) - (2)].list); + n->def = (PGNode *)(yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 19437 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 89: + case 92: /* alter_table_cmd: alter_generic_options */ #line 408 "third_party/libpg_query/grammar/statements/alter_table.y" - { + { PGAlterTableCmd *n = makeNode(PGAlterTableCmd); n->subtype = PG_AT_GenericOptions; - n->def = (PGNode *)(yyvsp[(1) - (1)].list); + n->def = (PGNode *)(yyvsp[0].list); (yyval.node) = (PGNode *) n; - ;} + } +#line 19448 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 90: + case 93: /* alter_using: USING a_expr */ #line 418 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 19454 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 91: + case 94: /* alter_using: %empty */ #line 419 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 19460 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 92: + case 95: /* alter_generic_option_elem: generic_option_elem */ #line 425 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = (yyvsp[(1) - (1)].defelt); - ;} + { + (yyval.defelt) = (yyvsp[0].defelt); + } +#line 19468 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 93: + case 96: /* alter_generic_option_elem: SET generic_option_elem */ #line 429 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = (yyvsp[(2) - (2)].defelt); + { + (yyval.defelt) = (yyvsp[0].defelt); (yyval.defelt)->defaction = PG_DEFELEM_SET; - ;} + } +#line 19477 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 94: + case 97: /* alter_generic_option_elem: ADD_P generic_option_elem */ #line 434 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = (yyvsp[(2) - (2)].defelt); + { + (yyval.defelt) = (yyvsp[0].defelt); (yyval.defelt)->defaction = PG_DEFELEM_ADD; - ;} + } +#line 19486 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 95: + case 98: /* alter_generic_option_elem: DROP generic_option_name */ #line 439 "third_party/libpg_query/grammar/statements/alter_table.y" - { - (yyval.defelt) = makeDefElemExtended(NULL, (yyvsp[(2) - (2)].str), NULL, DEFELEM_DROP, (yylsp[(2) - (2)])); - ;} + { + (yyval.defelt) = makeDefElemExtended(NULL, (yyvsp[0].str), NULL, DEFELEM_DROP, (yylsp[0])); + } +#line 19494 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 96: + case 99: /* alter_table_cmds: alter_table_cmd */ #line 446 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 19500 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 97: + case 100: /* alter_table_cmds: alter_table_cmds ',' alter_table_cmd */ #line 447 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 19506 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 98: + case 101: /* alter_generic_options: OPTIONS '(' alter_generic_option_list ')' */ #line 452 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 19512 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 99: + case 102: /* opt_set_data: SET DATA_P */ #line 456 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.ival) = 1; ;} + { (yyval.ival) = 1; } +#line 19518 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 100: + case 103: /* opt_set_data: SET */ #line 457 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 19524 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 101: + case 104: /* opt_set_data: %empty */ #line 458 "third_party/libpg_query/grammar/statements/alter_table.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 19530 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 102: + case 105: /* DeallocateStmt: DEALLOCATE name */ #line 8 "third_party/libpg_query/grammar/statements/deallocate.y" - { + { PGDeallocateStmt *n = makeNode(PGDeallocateStmt); - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *) n; - ;} + } +#line 19540 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 103: + case 106: /* DeallocateStmt: DEALLOCATE PREPARE name */ #line 14 "third_party/libpg_query/grammar/statements/deallocate.y" - { + { PGDeallocateStmt *n = makeNode(PGDeallocateStmt); - n->name = (yyvsp[(3) - (3)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *) n; - ;} + } +#line 19550 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 104: + case 107: /* DeallocateStmt: DEALLOCATE ALL */ #line 20 "third_party/libpg_query/grammar/statements/deallocate.y" - { + { PGDeallocateStmt *n = makeNode(PGDeallocateStmt); n->name = NULL; (yyval.node) = (PGNode *) n; - ;} + } +#line 19560 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 105: + case 108: /* DeallocateStmt: DEALLOCATE PREPARE ALL */ #line 26 "third_party/libpg_query/grammar/statements/deallocate.y" - { + { PGDeallocateStmt *n = makeNode(PGDeallocateStmt); n->name = NULL; (yyval.node) = (PGNode *) n; - ;} + } +#line 19570 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 106: + case 109: /* qualified_name: ColIdOrString */ #line 10 "third_party/libpg_query/grammar/statements/common.y" - { - (yyval.range) = makeRangeVar(NULL, (yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + { + (yyval.range) = makeRangeVar(NULL, (yyvsp[0].str), (yylsp[0])); + } +#line 19578 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 107: + case 110: /* qualified_name: ColId indirection */ #line 14 "third_party/libpg_query/grammar/statements/common.y" - { - check_qualified_name((yyvsp[(2) - (2)].list), yyscanner); - (yyval.range) = makeRangeVar(NULL, NULL, (yylsp[(1) - (2)])); - switch (list_length((yyvsp[(2) - (2)].list))) + { + check_qualified_name((yyvsp[0].list), yyscanner); + (yyval.range) = makeRangeVar(NULL, NULL, (yylsp[-1])); + switch (list_length((yyvsp[0].list))) { case 1: (yyval.range)->catalogname = NULL; - (yyval.range)->schemaname = (yyvsp[(1) - (2)].str); - (yyval.range)->relname = strVal(linitial((yyvsp[(2) - (2)].list))); + (yyval.range)->schemaname = (yyvsp[-1].str); + (yyval.range)->relname = strVal(linitial((yyvsp[0].list))); break; case 2: - (yyval.range)->catalogname = (yyvsp[(1) - (2)].str); - (yyval.range)->schemaname = strVal(linitial((yyvsp[(2) - (2)].list))); - (yyval.range)->relname = strVal(lsecond((yyvsp[(2) - (2)].list))); + (yyval.range)->catalogname = (yyvsp[-1].str); + (yyval.range)->schemaname = strVal(linitial((yyvsp[0].list))); + (yyval.range)->relname = strVal(lsecond((yyvsp[0].list))); break; case 3: default: ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("improper qualified name (too many dotted names): %s", - NameListToString(lcons(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].list)))), - parser_errposition((yylsp[(1) - (2)])))); + NameListToString(lcons(makeString((yyvsp[-1].str)), (yyvsp[0].list)))), + parser_errposition((yylsp[-1])))); break; } - ;} + } +#line 19608 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 108: + case 111: /* ColId: IDENT */ #line 44 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19614 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 109: + case 112: /* ColId: unreserved_keyword */ #line 45 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 19620 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 110: + case 113: /* ColId: col_name_keyword */ #line 46 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 19626 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 111: + case 114: /* ColIdOrString: ColId */ #line 50 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19632 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 112: + case 115: /* ColIdOrString: SCONST */ #line 51 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19638 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 113: + case 116: /* Sconst: SCONST */ #line 55 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19644 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 114: + case 117: /* indirection: indirection_el */ #line 59 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 19650 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 115: + case 118: /* indirection: indirection indirection_el */ #line 60 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 19656 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 116: + case 119: /* indirection_el: '.' attr_name */ #line 65 "third_party/libpg_query/grammar/statements/common.y" - { - (yyval.node) = (PGNode *) makeString((yyvsp[(2) - (2)].str)); - ;} + { + (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); + } +#line 19664 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 117: + case 120: /* attr_name: ColLabel */ #line 70 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19670 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 118: + case 121: /* ColLabel: IDENT */ #line 75 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 19676 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 119: + case 122: /* ColLabel: other_keyword */ #line 76 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 19682 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 120: + case 123: /* ColLabel: unreserved_keyword */ #line 77 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 19688 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 121: + case 124: /* ColLabel: reserved_keyword */ #line 78 "third_party/libpg_query/grammar/statements/common.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 19694 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 122: + case 125: /* RenameStmt: ALTER SCHEMA name RENAME TO name */ #line 7 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_SCHEMA; - n->subname = (yyvsp[(3) - (6)].str); - n->newname = (yyvsp[(6) - (6)].str); + n->subname = (yyvsp[-3].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19707 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 123: + case 126: /* RenameStmt: ALTER TABLE relation_expr RENAME TO name */ #line 16 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(3) - (6)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(6) - (6)].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19721 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 124: + case 127: /* RenameStmt: ALTER TABLE IF_P EXISTS relation_expr RENAME TO name */ #line 26 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(5) - (8)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(8) - (8)].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19735 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 125: + case 128: /* RenameStmt: ALTER SEQUENCE qualified_name RENAME TO name */ #line 36 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_SEQUENCE; - n->relation = (yyvsp[(3) - (6)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(6) - (6)].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19749 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 126: + case 129: /* RenameStmt: ALTER SEQUENCE IF_P EXISTS qualified_name RENAME TO name */ #line 46 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_SEQUENCE; - n->relation = (yyvsp[(5) - (8)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(8) - (8)].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19763 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 127: + case 130: /* RenameStmt: ALTER VIEW qualified_name RENAME TO name */ #line 56 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_VIEW; - n->relation = (yyvsp[(3) - (6)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(6) - (6)].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19777 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 128: + case 131: /* RenameStmt: ALTER VIEW IF_P EXISTS qualified_name RENAME TO name */ #line 66 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_VIEW; - n->relation = (yyvsp[(5) - (8)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(8) - (8)].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19791 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 129: + case 132: /* RenameStmt: ALTER INDEX qualified_name RENAME TO name */ #line 76 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_INDEX; - n->relation = (yyvsp[(3) - (6)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(6) - (6)].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19805 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 130: + case 133: /* RenameStmt: ALTER INDEX IF_P EXISTS qualified_name RENAME TO name */ #line 86 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_INDEX; - n->relation = (yyvsp[(5) - (8)].range); + n->relation = (yyvsp[-3].range); n->subname = NULL; - n->newname = (yyvsp[(8) - (8)].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19819 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 131: + case 134: /* RenameStmt: ALTER TABLE relation_expr RENAME opt_column name TO name */ #line 96 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_COLUMN; n->relationType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(3) - (8)].range); - n->subname = (yyvsp[(6) - (8)].str); - n->newname = (yyvsp[(8) - (8)].str); + n->relation = (yyvsp[-5].range); + n->subname = (yyvsp[-2].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19834 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 132: + case 135: /* RenameStmt: ALTER TABLE IF_P EXISTS relation_expr RENAME opt_column name TO name */ #line 107 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_COLUMN; n->relationType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(5) - (10)].range); - n->subname = (yyvsp[(8) - (10)].str); - n->newname = (yyvsp[(10) - (10)].str); + n->relation = (yyvsp[-5].range); + n->subname = (yyvsp[-2].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19849 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 133: + case 136: /* RenameStmt: ALTER TABLE relation_expr RENAME CONSTRAINT name TO name */ #line 118 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_TABCONSTRAINT; - n->relation = (yyvsp[(3) - (8)].range); - n->subname = (yyvsp[(6) - (8)].str); - n->newname = (yyvsp[(8) - (8)].str); + n->relation = (yyvsp[-5].range); + n->subname = (yyvsp[-2].str); + n->newname = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 19863 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 134: + case 137: /* RenameStmt: ALTER TABLE IF_P EXISTS relation_expr RENAME CONSTRAINT name TO name */ #line 128 "third_party/libpg_query/grammar/statements/rename.y" - { + { PGRenameStmt *n = makeNode(PGRenameStmt); n->renameType = PG_OBJECT_TABCONSTRAINT; - n->relation = (yyvsp[(5) - (10)].range); - n->subname = (yyvsp[(8) - (10)].str); - n->newname = (yyvsp[(10) - (10)].str); + n->relation = (yyvsp[-5].range); + n->subname = (yyvsp[-2].str); + n->newname = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 19877 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 135: + case 138: /* opt_column: COLUMN */ #line 140 "third_party/libpg_query/grammar/statements/rename.y" - { (yyval.ival) = COLUMN; ;} + { (yyval.ival) = COLUMN; } +#line 19883 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 136: + case 139: /* opt_column: %empty */ #line 141 "third_party/libpg_query/grammar/statements/rename.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 19889 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 137: + case 140: /* InsertStmt: opt_with_clause INSERT opt_or_action INTO insert_target opt_by_name_or_position insert_rest opt_on_conflict returning_clause */ #line 11 "third_party/libpg_query/grammar/statements/insert.y" - { - (yyvsp[(7) - (9)].istmt)->relation = (yyvsp[(5) - (9)].range); - (yyvsp[(7) - (9)].istmt)->onConflictAlias = (yyvsp[(3) - (9)].onconflictshorthand); - (yyvsp[(7) - (9)].istmt)->onConflictClause = (yyvsp[(8) - (9)].onconflict); - (yyvsp[(7) - (9)].istmt)->returningList = (yyvsp[(9) - (9)].list); - (yyvsp[(7) - (9)].istmt)->withClause = (yyvsp[(1) - (9)].with); - (yyvsp[(7) - (9)].istmt)->insert_column_order = (yyvsp[(6) - (9)].bynameorposition); - (yyval.node) = (PGNode *) (yyvsp[(7) - (9)].istmt); - ;} - break; - - case 138: + { + (yyvsp[-2].istmt)->relation = (yyvsp[-4].range); + (yyvsp[-2].istmt)->onConflictAlias = (yyvsp[-6].onconflictshorthand); + (yyvsp[-2].istmt)->onConflictClause = (yyvsp[-1].onconflict); + (yyvsp[-2].istmt)->returningList = (yyvsp[0].list); + (yyvsp[-2].istmt)->withClause = (yyvsp[-8].with); + (yyvsp[-2].istmt)->insert_column_order = (yyvsp[-3].bynameorposition); + (yyval.node) = (PGNode *) (yyvsp[-2].istmt); + } +#line 19903 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 141: /* insert_rest: SelectStmt */ #line 24 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.istmt) = makeNode(PGInsertStmt); (yyval.istmt)->cols = NIL; - (yyval.istmt)->selectStmt = (yyvsp[(1) - (1)].node); - ;} + (yyval.istmt)->selectStmt = (yyvsp[0].node); + } +#line 19913 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 139: + case 142: /* insert_rest: OVERRIDING override_kind VALUE_P SelectStmt */ #line 30 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.istmt) = makeNode(PGInsertStmt); (yyval.istmt)->cols = NIL; - (yyval.istmt)->override = (yyvsp[(2) - (4)].override); - (yyval.istmt)->selectStmt = (yyvsp[(4) - (4)].node); - ;} + (yyval.istmt)->override = (yyvsp[-2].override); + (yyval.istmt)->selectStmt = (yyvsp[0].node); + } +#line 19924 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 140: + case 143: /* insert_rest: '(' insert_column_list ')' SelectStmt */ #line 37 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.istmt) = makeNode(PGInsertStmt); - (yyval.istmt)->cols = (yyvsp[(2) - (4)].list); - (yyval.istmt)->selectStmt = (yyvsp[(4) - (4)].node); - ;} + (yyval.istmt)->cols = (yyvsp[-2].list); + (yyval.istmt)->selectStmt = (yyvsp[0].node); + } +#line 19934 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 141: + case 144: /* insert_rest: '(' insert_column_list ')' OVERRIDING override_kind VALUE_P SelectStmt */ #line 43 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.istmt) = makeNode(PGInsertStmt); - (yyval.istmt)->cols = (yyvsp[(2) - (7)].list); - (yyval.istmt)->override = (yyvsp[(5) - (7)].override); - (yyval.istmt)->selectStmt = (yyvsp[(7) - (7)].node); - ;} + (yyval.istmt)->cols = (yyvsp[-5].list); + (yyval.istmt)->override = (yyvsp[-2].override); + (yyval.istmt)->selectStmt = (yyvsp[0].node); + } +#line 19945 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 142: + case 145: /* insert_rest: DEFAULT VALUES */ #line 50 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.istmt) = makeNode(PGInsertStmt); (yyval.istmt)->cols = NIL; (yyval.istmt)->selectStmt = NULL; - ;} + } +#line 19955 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 143: + case 146: /* insert_target: qualified_name */ #line 60 "third_party/libpg_query/grammar/statements/insert.y" - { - (yyval.range) = (yyvsp[(1) - (1)].range); - ;} + { + (yyval.range) = (yyvsp[0].range); + } +#line 19963 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 144: + case 147: /* insert_target: qualified_name AS ColId */ #line 64 "third_party/libpg_query/grammar/statements/insert.y" - { - (yyvsp[(1) - (3)].range)->alias = makeAlias((yyvsp[(3) - (3)].str), NIL); - (yyval.range) = (yyvsp[(1) - (3)].range); - ;} + { + (yyvsp[-2].range)->alias = makeAlias((yyvsp[0].str), NIL); + (yyval.range) = (yyvsp[-2].range); + } +#line 19972 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 145: + case 148: /* opt_by_name_or_position: BY NAME_P */ #line 71 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.bynameorposition) = PG_INSERT_BY_NAME; ;} + { (yyval.bynameorposition) = PG_INSERT_BY_NAME; } +#line 19978 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 146: + case 149: /* opt_by_name_or_position: BY POSITION */ #line 72 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.bynameorposition) = PG_INSERT_BY_POSITION; ;} + { (yyval.bynameorposition) = PG_INSERT_BY_POSITION; } +#line 19984 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 147: + case 150: /* opt_by_name_or_position: %empty */ #line 73 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.bynameorposition) = PG_INSERT_BY_POSITION; ;} + { (yyval.bynameorposition) = PG_INSERT_BY_POSITION; } +#line 19990 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 148: + case 151: /* opt_conf_expr: '(' index_params ')' where_clause */ #line 78 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.infer) = makeNode(PGInferClause); - (yyval.infer)->indexElems = (yyvsp[(2) - (4)].list); - (yyval.infer)->whereClause = (yyvsp[(4) - (4)].node); + (yyval.infer)->indexElems = (yyvsp[-2].list); + (yyval.infer)->whereClause = (yyvsp[0].node); (yyval.infer)->conname = NULL; - (yyval.infer)->location = (yylsp[(1) - (4)]); - ;} + (yyval.infer)->location = (yylsp[-3]); + } +#line 20002 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 149: + case 152: /* opt_conf_expr: ON CONSTRAINT name */ #line 87 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.infer) = makeNode(PGInferClause); (yyval.infer)->indexElems = NIL; (yyval.infer)->whereClause = NULL; - (yyval.infer)->conname = (yyvsp[(3) - (3)].str); - (yyval.infer)->location = (yylsp[(1) - (3)]); - ;} + (yyval.infer)->conname = (yyvsp[0].str); + (yyval.infer)->location = (yylsp[-2]); + } +#line 20014 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 150: + case 153: /* opt_conf_expr: %empty */ #line 95 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.infer) = NULL; - ;} + } +#line 20022 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 151: + case 154: /* opt_with_clause: with_clause */ #line 102 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.with) = (yyvsp[(1) - (1)].with); ;} + { (yyval.with) = (yyvsp[0].with); } +#line 20028 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 152: + case 155: /* opt_with_clause: %empty */ #line 103 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.with) = NULL; ;} + { (yyval.with) = NULL; } +#line 20034 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 153: + case 156: /* insert_column_item: ColId opt_indirection */ #line 109 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.target) = makeNode(PGResTarget); - (yyval.target)->name = (yyvsp[(1) - (2)].str); - (yyval.target)->indirection = check_indirection((yyvsp[(2) - (2)].list), yyscanner); + (yyval.target)->name = (yyvsp[-1].str); + (yyval.target)->indirection = check_indirection((yyvsp[0].list), yyscanner); (yyval.target)->val = NULL; - (yyval.target)->location = (yylsp[(1) - (2)]); - ;} + (yyval.target)->location = (yylsp[-1]); + } +#line 20046 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 154: + case 157: /* set_clause: set_target '=' a_expr */ #line 121 "third_party/libpg_query/grammar/statements/insert.y" - { - (yyvsp[(1) - (3)].target)->val = (PGNode *) (yyvsp[(3) - (3)].node); - (yyval.list) = list_make1((yyvsp[(1) - (3)].target)); - ;} + { + (yyvsp[-2].target)->val = (PGNode *) (yyvsp[0].node); + (yyval.list) = list_make1((yyvsp[-2].target)); + } +#line 20055 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 155: + case 158: /* set_clause: '(' set_target_list ')' '=' a_expr */ #line 126 "third_party/libpg_query/grammar/statements/insert.y" - { - int ncolumns = list_length((yyvsp[(2) - (5)].list)); + { + int ncolumns = list_length((yyvsp[-3].list)); int i = 1; PGListCell *col_cell; /* Create a PGMultiAssignRef source for each target */ - foreach(col_cell, (yyvsp[(2) - (5)].list)) + foreach(col_cell, (yyvsp[-3].list)) { PGResTarget *res_col = (PGResTarget *) lfirst(col_cell); PGMultiAssignRef *r = makeNode(PGMultiAssignRef); - r->source = (PGNode *) (yyvsp[(5) - (5)].node); + r->source = (PGNode *) (yyvsp[0].node); r->colno = i; r->ncolumns = ncolumns; res_col->val = (PGNode *) r; i++; } - (yyval.list) = (yyvsp[(2) - (5)].list); - ;} + (yyval.list) = (yyvsp[-3].list); + } +#line 20080 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 156: + case 159: /* opt_or_action: OR REPLACE */ #line 151 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflictshorthand) = PG_ONCONFLICT_ALIAS_REPLACE; - ;} + } +#line 20088 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 157: + case 160: /* opt_or_action: OR IGNORE_P */ #line 156 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflictshorthand) = PG_ONCONFLICT_ALIAS_IGNORE; - ;} + } +#line 20096 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 158: + case 161: /* opt_or_action: %empty */ #line 160 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflictshorthand) = PG_ONCONFLICT_ALIAS_NONE; - ;} + } +#line 20104 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 159: + case 162: /* opt_on_conflict: ON CONFLICT opt_conf_expr DO UPDATE SET set_clause_list_opt_comma where_clause */ #line 167 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflict) = makeNode(PGOnConflictClause); (yyval.onconflict)->action = PG_ONCONFLICT_UPDATE; - (yyval.onconflict)->infer = (yyvsp[(3) - (8)].infer); - (yyval.onconflict)->targetList = (yyvsp[(7) - (8)].list); - (yyval.onconflict)->whereClause = (yyvsp[(8) - (8)].node); - (yyval.onconflict)->location = (yylsp[(1) - (8)]); - ;} + (yyval.onconflict)->infer = (yyvsp[-5].infer); + (yyval.onconflict)->targetList = (yyvsp[-1].list); + (yyval.onconflict)->whereClause = (yyvsp[0].node); + (yyval.onconflict)->location = (yylsp[-7]); + } +#line 20117 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 160: + case 163: /* opt_on_conflict: ON CONFLICT opt_conf_expr DO NOTHING */ #line 177 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflict) = makeNode(PGOnConflictClause); (yyval.onconflict)->action = PG_ONCONFLICT_NOTHING; - (yyval.onconflict)->infer = (yyvsp[(3) - (5)].infer); + (yyval.onconflict)->infer = (yyvsp[-2].infer); (yyval.onconflict)->targetList = NIL; (yyval.onconflict)->whereClause = NULL; - (yyval.onconflict)->location = (yylsp[(1) - (5)]); - ;} + (yyval.onconflict)->location = (yylsp[-4]); + } +#line 20130 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 161: + case 164: /* opt_on_conflict: %empty */ #line 186 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.onconflict) = NULL; - ;} + } +#line 20138 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 162: + case 165: /* index_elem: ColId opt_collate opt_class opt_asc_desc opt_nulls_order */ #line 193 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.ielem) = makeNode(PGIndexElem); - (yyval.ielem)->name = (yyvsp[(1) - (5)].str); + (yyval.ielem)->name = (yyvsp[-4].str); (yyval.ielem)->expr = NULL; (yyval.ielem)->indexcolname = NULL; - (yyval.ielem)->collation = (yyvsp[(2) - (5)].list); - (yyval.ielem)->opclass = (yyvsp[(3) - (5)].list); - (yyval.ielem)->ordering = (yyvsp[(4) - (5)].sortorder); - (yyval.ielem)->nulls_ordering = (yyvsp[(5) - (5)].nullorder); - ;} + (yyval.ielem)->collation = (yyvsp[-3].list); + (yyval.ielem)->opclass = (yyvsp[-2].list); + (yyval.ielem)->ordering = (yyvsp[-1].sortorder); + (yyval.ielem)->nulls_ordering = (yyvsp[0].nullorder); + } +#line 20153 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 163: + case 166: /* index_elem: func_expr_windowless opt_collate opt_class opt_asc_desc opt_nulls_order */ #line 204 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.ielem) = makeNode(PGIndexElem); (yyval.ielem)->name = NULL; - (yyval.ielem)->expr = (yyvsp[(1) - (5)].node); + (yyval.ielem)->expr = (yyvsp[-4].node); (yyval.ielem)->indexcolname = NULL; - (yyval.ielem)->collation = (yyvsp[(2) - (5)].list); - (yyval.ielem)->opclass = (yyvsp[(3) - (5)].list); - (yyval.ielem)->ordering = (yyvsp[(4) - (5)].sortorder); - (yyval.ielem)->nulls_ordering = (yyvsp[(5) - (5)].nullorder); - ;} + (yyval.ielem)->collation = (yyvsp[-3].list); + (yyval.ielem)->opclass = (yyvsp[-2].list); + (yyval.ielem)->ordering = (yyvsp[-1].sortorder); + (yyval.ielem)->nulls_ordering = (yyvsp[0].nullorder); + } +#line 20168 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 164: + case 167: /* index_elem: '(' a_expr ')' opt_collate opt_class opt_asc_desc opt_nulls_order */ #line 215 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.ielem) = makeNode(PGIndexElem); (yyval.ielem)->name = NULL; - (yyval.ielem)->expr = (yyvsp[(2) - (7)].node); + (yyval.ielem)->expr = (yyvsp[-5].node); (yyval.ielem)->indexcolname = NULL; - (yyval.ielem)->collation = (yyvsp[(4) - (7)].list); - (yyval.ielem)->opclass = (yyvsp[(5) - (7)].list); - (yyval.ielem)->ordering = (yyvsp[(6) - (7)].sortorder); - (yyval.ielem)->nulls_ordering = (yyvsp[(7) - (7)].nullorder); - ;} + (yyval.ielem)->collation = (yyvsp[-3].list); + (yyval.ielem)->opclass = (yyvsp[-2].list); + (yyval.ielem)->ordering = (yyvsp[-1].sortorder); + (yyval.ielem)->nulls_ordering = (yyvsp[0].nullorder); + } +#line 20183 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 165: + case 168: /* returning_clause: RETURNING target_list */ #line 229 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20189 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 166: + case 169: /* returning_clause: %empty */ #line 230 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 20195 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 167: + case 170: /* override_kind: USER */ #line 236 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.override) = PG_OVERRIDING_USER_VALUE; ;} + { (yyval.override) = PG_OVERRIDING_USER_VALUE; } +#line 20201 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 168: + case 171: /* override_kind: SYSTEM_P */ #line 237 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.override) = OVERRIDING_SYSTEM_VALUE; ;} + { (yyval.override) = OVERRIDING_SYSTEM_VALUE; } +#line 20207 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 169: + case 172: /* set_target_list: set_target */ #line 242 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].target)); ;} + { (yyval.list) = list_make1((yyvsp[0].target)); } +#line 20213 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 170: + case 173: /* set_target_list: set_target_list ',' set_target */ #line 243 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list),(yyvsp[(3) - (3)].target)); ;} + { (yyval.list) = lappend((yyvsp[-2].list),(yyvsp[0].target)); } +#line 20219 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 171: + case 174: /* opt_collate: COLLATE any_name */ #line 249 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20225 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 172: + case 175: /* opt_collate: %empty */ #line 250 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 20231 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 173: + case 176: /* opt_class: any_name */ #line 254 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20237 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 174: + case 177: /* opt_class: %empty */ #line 255 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 20243 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 175: + case 178: /* insert_column_list: insert_column_item */ #line 261 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].target)); ;} + { (yyval.list) = list_make1((yyvsp[0].target)); } +#line 20249 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 176: + case 179: /* insert_column_list: insert_column_list ',' insert_column_item */ #line 263 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].target)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].target)); } +#line 20255 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 177: + case 180: /* set_clause_list: set_clause */ #line 268 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20261 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 178: + case 181: /* set_clause_list: set_clause_list ',' set_clause */ #line 269 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = list_concat((yyvsp[(1) - (3)].list),(yyvsp[(3) - (3)].list)); ;} + { (yyval.list) = list_concat((yyvsp[-2].list),(yyvsp[0].list)); } +#line 20267 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 179: + case 182: /* set_clause_list_opt_comma: set_clause_list */ #line 273 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20273 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 180: + case 183: /* set_clause_list_opt_comma: set_clause_list ',' */ #line 274 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 20279 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 181: + case 184: /* index_params: index_elem */ #line 277 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].ielem)); ;} + { (yyval.list) = list_make1((yyvsp[0].ielem)); } +#line 20285 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 182: + case 185: /* index_params: index_params ',' index_elem */ #line 278 "third_party/libpg_query/grammar/statements/insert.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].ielem)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].ielem)); } +#line 20291 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 183: + case 186: /* set_target: ColId opt_indirection */ #line 284 "third_party/libpg_query/grammar/statements/insert.y" - { + { (yyval.target) = makeNode(PGResTarget); - (yyval.target)->name = (yyvsp[(1) - (2)].str); - (yyval.target)->indirection = check_indirection((yyvsp[(2) - (2)].list), yyscanner); + (yyval.target)->name = (yyvsp[-1].str); + (yyval.target)->indirection = check_indirection((yyvsp[0].list), yyscanner); (yyval.target)->val = NULL; /* upper production sets this */ - (yyval.target)->location = (yylsp[(1) - (2)]); - ;} + (yyval.target)->location = (yylsp[-1]); + } +#line 20303 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 184: + case 187: /* CreateTypeStmt: CREATE_P TYPE_P qualified_name AS ENUM_P select_with_parens */ #line 8 "third_party/libpg_query/grammar/statements/create_type.y" - { + { PGCreateTypeStmt *n = makeNode(PGCreateTypeStmt); - n->typeName = (yyvsp[(3) - (6)].range); + n->typeName = (yyvsp[-3].range); n->kind = PG_NEWTYPE_ENUM; - n->query = (yyvsp[(6) - (6)].node); + n->query = (yyvsp[0].node); n->vals = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20316 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 185: + case 188: /* CreateTypeStmt: CREATE_P TYPE_P qualified_name AS ENUM_P '(' opt_enum_val_list ')' */ #line 17 "third_party/libpg_query/grammar/statements/create_type.y" - { + { PGCreateTypeStmt *n = makeNode(PGCreateTypeStmt); - n->typeName = (yyvsp[(3) - (8)].range); + n->typeName = (yyvsp[-5].range); n->kind = PG_NEWTYPE_ENUM; - n->vals = (yyvsp[(7) - (8)].list); + n->vals = (yyvsp[-1].list); n->query = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20329 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 186: + case 189: /* CreateTypeStmt: CREATE_P TYPE_P qualified_name AS Typename */ #line 26 "third_party/libpg_query/grammar/statements/create_type.y" - { + { PGCreateTypeStmt *n = makeNode(PGCreateTypeStmt); - n->typeName = (yyvsp[(3) - (5)].range); + n->typeName = (yyvsp[-2].range); n->query = NULL; - auto name = std::string(reinterpret_cast((yyvsp[(5) - (5)].typnam)->names->tail->data.ptr_value)->val.str); + auto name = std::string(reinterpret_cast((yyvsp[0].typnam)->names->tail->data.ptr_value)->val.str); if (name == "enum") { n->kind = PG_NEWTYPE_ENUM; - n->vals = (yyvsp[(5) - (5)].typnam)->typmods; + n->vals = (yyvsp[0].typnam)->typmods; } else { n->kind = PG_NEWTYPE_ALIAS; - n->ofType = (yyvsp[(5) - (5)].typnam); + n->ofType = (yyvsp[0].typnam); } (yyval.node) = (PGNode *)n; - ;} + } +#line 20348 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 187: + case 190: /* opt_enum_val_list: enum_val_list */ #line 46 "third_party/libpg_query/grammar/statements/create_type.y" - { (yyval.list) = (yyvsp[(1) - (1)].list);;} + { (yyval.list) = (yyvsp[0].list);} +#line 20354 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 188: + case 191: /* opt_enum_val_list: %empty */ #line 47 "third_party/libpg_query/grammar/statements/create_type.y" - {(yyval.list) = NIL;;} + {(yyval.list) = NIL;} +#line 20360 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 189: + case 192: /* enum_val_list: Sconst */ #line 51 "third_party/libpg_query/grammar/statements/create_type.y" - { - (yyval.list) = list_make1(makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)]))); - ;} + { + (yyval.list) = list_make1(makeStringConst((yyvsp[0].str), (yylsp[0]))); + } +#line 20368 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 190: + case 193: /* enum_val_list: enum_val_list ',' Sconst */ #line 55 "third_party/libpg_query/grammar/statements/create_type.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeStringConst((yyvsp[(3) - (3)].str), (yylsp[(3) - (3)]))); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), makeStringConst((yyvsp[0].str), (yylsp[0]))); + } +#line 20376 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 191: + case 194: /* PragmaStmt: PRAGMA_P ColId */ #line 8 "third_party/libpg_query/grammar/statements/pragma.y" - { + { PGPragmaStmt *n = makeNode(PGPragmaStmt); n->kind = PG_PRAGMA_TYPE_NOTHING; - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 20387 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 192: + case 195: /* PragmaStmt: PRAGMA_P ColId '=' var_list */ #line 15 "third_party/libpg_query/grammar/statements/pragma.y" - { + { PGPragmaStmt *n = makeNode(PGPragmaStmt); n->kind = PG_PRAGMA_TYPE_ASSIGNMENT; - n->name = (yyvsp[(2) - (4)].str); - n->args = (yyvsp[(4) - (4)].list); + n->name = (yyvsp[-2].str); + n->args = (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 20399 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 193: + case 196: /* PragmaStmt: PRAGMA_P ColId '(' func_arg_list ')' */ #line 23 "third_party/libpg_query/grammar/statements/pragma.y" - { + { PGPragmaStmt *n = makeNode(PGPragmaStmt); n->kind = PG_PRAGMA_TYPE_CALL; - n->name = (yyvsp[(2) - (5)].str); - n->args = (yyvsp[(4) - (5)].list); + n->name = (yyvsp[-3].str); + n->args = (yyvsp[-1].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 20411 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 194: + case 197: /* CreateSeqStmt: CREATE_P OptTemp SEQUENCE qualified_name OptSeqOptList */ #line 10 "third_party/libpg_query/grammar/statements/create_sequence.y" - { + { PGCreateSeqStmt *n = makeNode(PGCreateSeqStmt); - (yyvsp[(4) - (5)].range)->relpersistence = (yyvsp[(2) - (5)].ival); - n->sequence = (yyvsp[(4) - (5)].range); - n->options = (yyvsp[(5) - (5)].list); + (yyvsp[-1].range)->relpersistence = (yyvsp[-3].ival); + n->sequence = (yyvsp[-1].range); + n->options = (yyvsp[0].list); n->ownerId = InvalidOid; n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 20425 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 195: + case 198: /* CreateSeqStmt: CREATE_P OptTemp SEQUENCE IF_P NOT EXISTS qualified_name OptSeqOptList */ #line 20 "third_party/libpg_query/grammar/statements/create_sequence.y" - { + { PGCreateSeqStmt *n = makeNode(PGCreateSeqStmt); - (yyvsp[(7) - (8)].range)->relpersistence = (yyvsp[(2) - (8)].ival); - n->sequence = (yyvsp[(7) - (8)].range); - n->options = (yyvsp[(8) - (8)].list); + (yyvsp[-1].range)->relpersistence = (yyvsp[-6].ival); + n->sequence = (yyvsp[-1].range); + n->options = (yyvsp[0].list); n->ownerId = InvalidOid; n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 20439 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 196: + case 199: /* CreateSeqStmt: CREATE_P OR REPLACE OptTemp SEQUENCE qualified_name OptSeqOptList */ #line 30 "third_party/libpg_query/grammar/statements/create_sequence.y" - { + { PGCreateSeqStmt *n = makeNode(PGCreateSeqStmt); - (yyvsp[(6) - (7)].range)->relpersistence = (yyvsp[(4) - (7)].ival); - n->sequence = (yyvsp[(6) - (7)].range); - n->options = (yyvsp[(7) - (7)].list); + (yyvsp[-1].range)->relpersistence = (yyvsp[-3].ival); + n->sequence = (yyvsp[-1].range); + n->options = (yyvsp[0].list); n->ownerId = InvalidOid; n->onconflict = PG_REPLACE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 20453 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 197: + case 200: /* OptSeqOptList: SeqOptList */ #line 42 "third_party/libpg_query/grammar/statements/create_sequence.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 20459 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 198: + case 201: /* OptSeqOptList: %empty */ #line 43 "third_party/libpg_query/grammar/statements/create_sequence.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 20465 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 202: /* CreateSecretStmt: CREATE_P opt_persist SECRET opt_secret_name opt_storage_specifier '(' copy_generic_opt_list ')' */ +#line 8 "third_party/libpg_query/grammar/statements/create_secret.y" + { + PGCreateSecretStmt *n = makeNode(PGCreateSecretStmt); + n->persist_type = (yyvsp[-6].str); + n->secret_name = (yyvsp[-4].str); + n->secret_storage = (yyvsp[-3].str); + n->options = (yyvsp[-1].list); + n->onconflict = PG_ERROR_ON_CONFLICT; + (yyval.node) = (PGNode *)n; + } +#line 20479 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 203: /* CreateSecretStmt: CREATE_P opt_persist SECRET IF_P NOT EXISTS opt_secret_name opt_storage_specifier '(' copy_generic_opt_list ')' */ +#line 18 "third_party/libpg_query/grammar/statements/create_secret.y" + { + PGCreateSecretStmt *n = makeNode(PGCreateSecretStmt); + n->persist_type = (yyvsp[-9].str); + n->secret_name = (yyvsp[-4].str); + n->secret_storage = (yyvsp[-3].str); + n->options = (yyvsp[-1].list); + n->onconflict = PG_IGNORE_ON_CONFLICT; + (yyval.node) = (PGNode *)n; + } +#line 20493 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 204: /* CreateSecretStmt: CREATE_P OR REPLACE opt_persist SECRET opt_secret_name opt_storage_specifier '(' copy_generic_opt_list ')' */ +#line 28 "third_party/libpg_query/grammar/statements/create_secret.y" + { + PGCreateSecretStmt *n = makeNode(PGCreateSecretStmt); + n->persist_type = (yyvsp[-6].str); + n->secret_name = (yyvsp[-4].str); + n->secret_storage = (yyvsp[-3].str); + n->options = (yyvsp[-1].list); + n->onconflict = PG_REPLACE_ON_CONFLICT; + (yyval.node) = (PGNode *)n; + } +#line 20507 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 205: /* opt_secret_name: %empty */ +#line 40 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = NULL; } +#line 20513 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 206: /* opt_secret_name: ColId */ +#line 41 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = (yyvsp[0].str); } +#line 20519 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 207: /* opt_persist: %empty */ +#line 45 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = pstrdup("default"); } +#line 20525 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 208: /* opt_persist: TEMPORARY */ +#line 46 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = pstrdup("temporary"); } +#line 20531 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 209: /* opt_persist: PERSISTENT */ +#line 47 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = pstrdup("persistent"); } +#line 20537 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 199: + case 210: /* opt_storage_specifier: %empty */ +#line 51 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = pstrdup(""); } +#line 20543 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 211: /* opt_storage_specifier: IN_P IDENT */ +#line 52 "third_party/libpg_query/grammar/statements/create_secret.y" + { (yyval.str) = (yyvsp[0].str); } +#line 20549 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 212: /* ExecuteStmt: EXECUTE name execute_param_clause */ #line 8 "third_party/libpg_query/grammar/statements/execute.y" - { + { PGExecuteStmt *n = makeNode(PGExecuteStmt); - n->name = (yyvsp[(2) - (3)].str); - n->params = (yyvsp[(3) - (3)].list); + n->name = (yyvsp[-1].str); + n->params = (yyvsp[0].list); (yyval.node) = (PGNode *) n; - ;} + } +#line 20560 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 200: + case 213: /* ExecuteStmt: CREATE_P OptTemp TABLE create_as_target AS EXECUTE name execute_param_clause opt_with_data */ #line 16 "third_party/libpg_query/grammar/statements/execute.y" - { + { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); PGExecuteStmt *n = makeNode(PGExecuteStmt); - n->name = (yyvsp[(7) - (9)].str); - n->params = (yyvsp[(8) - (9)].list); + n->name = (yyvsp[-2].str); + n->params = (yyvsp[-1].list); ctas->query = (PGNode *) n; - ctas->into = (yyvsp[(4) - (9)].into); + ctas->into = (yyvsp[-5].into); ctas->relkind = PG_OBJECT_TABLE; ctas->is_select_into = false; ctas->onconflict = PG_ERROR_ON_CONFLICT; /* cram additional flags into the PGIntoClause */ - (yyvsp[(4) - (9)].into)->rel->relpersistence = (yyvsp[(2) - (9)].ival); - (yyvsp[(4) - (9)].into)->skipData = !((yyvsp[(9) - (9)].boolean)); + (yyvsp[-5].into)->rel->relpersistence = (yyvsp[-7].ival); + (yyvsp[-5].into)->skipData = !((yyvsp[0].boolean)); (yyval.node) = (PGNode *) ctas; - ;} + } +#line 20580 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 201: + case 214: /* ExecuteStmt: CREATE_P OptTemp TABLE IF_P NOT EXISTS create_as_target AS EXECUTE name execute_param_clause opt_with_data */ #line 33 "third_party/libpg_query/grammar/statements/execute.y" - { + { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); PGExecuteStmt *n = makeNode(PGExecuteStmt); - n->name = (yyvsp[(10) - (12)].str); - n->params = (yyvsp[(11) - (12)].list); + n->name = (yyvsp[-2].str); + n->params = (yyvsp[-1].list); ctas->query = (PGNode *) n; - ctas->into = (yyvsp[(7) - (12)].into); + ctas->into = (yyvsp[-5].into); ctas->relkind = PG_OBJECT_TABLE; ctas->is_select_into = false; ctas->onconflict = PG_IGNORE_ON_CONFLICT; /* cram additional flags into the PGIntoClause */ - (yyvsp[(7) - (12)].into)->rel->relpersistence = (yyvsp[(2) - (12)].ival); - (yyvsp[(7) - (12)].into)->skipData = !((yyvsp[(12) - (12)].boolean)); + (yyvsp[-5].into)->rel->relpersistence = (yyvsp[-10].ival); + (yyvsp[-5].into)->skipData = !((yyvsp[0].boolean)); (yyval.node) = (PGNode *) ctas; - ;} + } +#line 20600 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 202: + case 215: /* execute_param_expr: a_expr */ #line 52 "third_party/libpg_query/grammar/statements/execute.y" - { - (yyval.node) = (yyvsp[(1) - (1)].node); - ;} + { + (yyval.node) = (yyvsp[0].node); + } +#line 20608 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 203: + case 216: /* execute_param_expr: param_name COLON_EQUALS a_expr */ #line 56 "third_party/libpg_query/grammar/statements/execute.y" - { + { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); - na->name = (yyvsp[(1) - (3)].str); - na->arg = (PGExpr *) (yyvsp[(3) - (3)].node); + na->name = (yyvsp[-2].str); + na->arg = (PGExpr *) (yyvsp[0].node); na->argnumber = -1; /* until determined */ - na->location = (yylsp[(1) - (3)]); + na->location = (yylsp[-2]); (yyval.node) = (PGNode *) na; - ;} + } +#line 20621 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 204: + case 217: /* execute_param_list: execute_param_expr */ #line 66 "third_party/libpg_query/grammar/statements/execute.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 20629 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 205: + case 218: /* execute_param_list: execute_param_list ',' execute_param_expr */ #line 70 "third_party/libpg_query/grammar/statements/execute.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 20637 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 206: + case 219: /* execute_param_clause: '(' execute_param_list ')' */ #line 75 "third_party/libpg_query/grammar/statements/execute.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 20643 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 207: + case 220: /* execute_param_clause: %empty */ #line 76 "third_party/libpg_query/grammar/statements/execute.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 20649 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 208: + case 221: /* AlterSeqStmt: ALTER SEQUENCE qualified_name SeqOptList */ #line 10 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { + { PGAlterSeqStmt *n = makeNode(PGAlterSeqStmt); - n->sequence = (yyvsp[(3) - (4)].range); - n->options = (yyvsp[(4) - (4)].list); + n->sequence = (yyvsp[-1].range); + n->options = (yyvsp[0].list); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 20661 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 209: + case 222: /* AlterSeqStmt: ALTER SEQUENCE IF_P EXISTS qualified_name SeqOptList */ #line 18 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { + { PGAlterSeqStmt *n = makeNode(PGAlterSeqStmt); - n->sequence = (yyvsp[(5) - (6)].range); - n->options = (yyvsp[(6) - (6)].list); + n->sequence = (yyvsp[-1].range); + n->options = (yyvsp[0].list); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 20673 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 210: + case 223: /* SeqOptList: SeqOptElem */ #line 29 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); ;} + { (yyval.list) = list_make1((yyvsp[0].defelt)); } +#line 20679 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 211: + case 224: /* SeqOptList: SeqOptList SeqOptElem */ #line 30 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].defelt)); ;} + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].defelt)); } +#line 20685 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 212: + case 225: /* opt_with: WITH */ #line 34 "third_party/libpg_query/grammar/statements/alter_sequence.y" - {;} + {} +#line 20691 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 213: + case 226: /* opt_with: WITH_LA */ #line 35 "third_party/libpg_query/grammar/statements/alter_sequence.y" - {;} + {} +#line 20697 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 214: + case 227: /* opt_with: %empty */ #line 36 "third_party/libpg_query/grammar/statements/alter_sequence.y" - {;} + {} +#line 20703 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 215: + case 228: /* NumericOnly: FCONST */ #line 41 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.value) = makeFloat((yyvsp[(1) - (1)].str)); ;} + { (yyval.value) = makeFloat((yyvsp[0].str)); } +#line 20709 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 216: + case 229: /* NumericOnly: '+' FCONST */ #line 42 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.value) = makeFloat((yyvsp[(2) - (2)].str)); ;} + { (yyval.value) = makeFloat((yyvsp[0].str)); } +#line 20715 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 217: + case 230: /* NumericOnly: '-' FCONST */ #line 44 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.value) = makeFloat((yyvsp[(2) - (2)].str)); + { + (yyval.value) = makeFloat((yyvsp[0].str)); doNegateFloat((yyval.value)); - ;} + } +#line 20724 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 218: + case 231: /* NumericOnly: SignedIconst */ #line 48 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.value) = makeInteger((yyvsp[(1) - (1)].ival)); ;} + { (yyval.value) = makeInteger((yyvsp[0].ival)); } +#line 20730 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 219: + case 232: /* SeqOptElem: AS SimpleTypename */ #line 53 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("as", (PGNode *)(yyvsp[(2) - (2)].typnam), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("as", (PGNode *)(yyvsp[0].typnam), (yylsp[-1])); + } +#line 20738 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 220: + case 233: /* SeqOptElem: CACHE NumericOnly */ #line 57 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("cache", (PGNode *)(yyvsp[(2) - (2)].value), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("cache", (PGNode *)(yyvsp[0].value), (yylsp[-1])); + } +#line 20746 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 221: + case 234: /* SeqOptElem: CYCLE */ #line 61 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("cycle", (PGNode *)makeInteger(true), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("cycle", (PGNode *)makeInteger(true), (yylsp[0])); + } +#line 20754 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 222: + case 235: /* SeqOptElem: NO CYCLE */ #line 65 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("cycle", (PGNode *)makeInteger(false), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("cycle", (PGNode *)makeInteger(false), (yylsp[-1])); + } +#line 20762 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 223: + case 236: /* SeqOptElem: INCREMENT opt_by NumericOnly */ #line 69 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("increment", (PGNode *)(yyvsp[(3) - (3)].value), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("increment", (PGNode *)(yyvsp[0].value), (yylsp[-2])); + } +#line 20770 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 224: + case 237: /* SeqOptElem: MAXVALUE NumericOnly */ #line 73 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("maxvalue", (PGNode *)(yyvsp[(2) - (2)].value), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("maxvalue", (PGNode *)(yyvsp[0].value), (yylsp[-1])); + } +#line 20778 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 225: + case 238: /* SeqOptElem: MINVALUE NumericOnly */ #line 77 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("minvalue", (PGNode *)(yyvsp[(2) - (2)].value), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("minvalue", (PGNode *)(yyvsp[0].value), (yylsp[-1])); + } +#line 20786 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 226: + case 239: /* SeqOptElem: NO MAXVALUE */ #line 81 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("maxvalue", NULL, (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("maxvalue", NULL, (yylsp[-1])); + } +#line 20794 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 227: + case 240: /* SeqOptElem: NO MINVALUE */ #line 85 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("minvalue", NULL, (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("minvalue", NULL, (yylsp[-1])); + } +#line 20802 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 228: + case 241: /* SeqOptElem: OWNED BY any_name */ #line 89 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("owned_by", (PGNode *)(yyvsp[(3) - (3)].list), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("owned_by", (PGNode *)(yyvsp[0].list), (yylsp[-2])); + } +#line 20810 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 229: + case 242: /* SeqOptElem: SEQUENCE NAME_P any_name */ #line 93 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { + { /* not documented, only used by pg_dump */ - (yyval.defelt) = makeDefElem("sequence_name", (PGNode *)(yyvsp[(3) - (3)].list), (yylsp[(1) - (3)])); - ;} + (yyval.defelt) = makeDefElem("sequence_name", (PGNode *)(yyvsp[0].list), (yylsp[-2])); + } +#line 20819 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 230: + case 243: /* SeqOptElem: START opt_with NumericOnly */ #line 98 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("start", (PGNode *)(yyvsp[(3) - (3)].value), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("start", (PGNode *)(yyvsp[0].value), (yylsp[-2])); + } +#line 20827 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 231: + case 244: /* SeqOptElem: RESTART */ #line 102 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("restart", NULL, (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("restart", NULL, (yylsp[0])); + } +#line 20835 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 232: + case 245: /* SeqOptElem: RESTART opt_with NumericOnly */ #line 106 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { - (yyval.defelt) = makeDefElem("restart", (PGNode *)(yyvsp[(3) - (3)].value), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("restart", (PGNode *)(yyvsp[0].value), (yylsp[-2])); + } +#line 20843 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 233: + case 246: /* opt_by: BY */ #line 112 "third_party/libpg_query/grammar/statements/alter_sequence.y" - {;} + {} +#line 20849 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 234: + case 247: /* opt_by: %empty */ #line 113 "third_party/libpg_query/grammar/statements/alter_sequence.y" - {;} + {} +#line 20855 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 235: + case 248: /* SignedIconst: Iconst */ #line 117 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.ival) = (yyvsp[(1) - (1)].ival); ;} + { (yyval.ival) = (yyvsp[0].ival); } +#line 20861 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 236: + case 249: /* SignedIconst: '+' Iconst */ #line 118 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.ival) = + (yyvsp[(2) - (2)].ival); ;} + { (yyval.ival) = + (yyvsp[0].ival); } +#line 20867 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 237: + case 250: /* SignedIconst: '-' Iconst */ #line 119 "third_party/libpg_query/grammar/statements/alter_sequence.y" - { (yyval.ival) = - (yyvsp[(2) - (2)].ival); ;} + { (yyval.ival) = - (yyvsp[0].ival); } +#line 20873 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 251: /* DropSecretStmt: DROP opt_persist SECRET ColId opt_storage_drop_specifier */ +#line 8 "third_party/libpg_query/grammar/statements/drop_secret.y" + { + PGDropSecretStmt *n = makeNode(PGDropSecretStmt); + n->persist_type = (yyvsp[-3].str); + n->secret_name = (yyvsp[-1].str); + n->secret_storage = (yyvsp[0].str); + n->missing_ok = false; + (yyval.node) = (PGNode *)n; + } +#line 20886 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 252: /* DropSecretStmt: DROP opt_persist SECRET IF_P EXISTS ColId opt_storage_drop_specifier */ +#line 17 "third_party/libpg_query/grammar/statements/drop_secret.y" + { + PGDropSecretStmt *n = makeNode(PGDropSecretStmt); + n->persist_type = (yyvsp[-5].str); + n->secret_name = (yyvsp[-1].str); + n->secret_storage = (yyvsp[0].str); + n->missing_ok = true; + (yyval.node) = (PGNode *)n; + } +#line 20899 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 253: /* opt_storage_drop_specifier: %empty */ +#line 28 "third_party/libpg_query/grammar/statements/drop_secret.y" + { (yyval.str) = pstrdup(""); } +#line 20905 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 238: + case 254: /* opt_storage_drop_specifier: FROM IDENT */ +#line 29 "third_party/libpg_query/grammar/statements/drop_secret.y" + { (yyval.str) = (yyvsp[0].str); } +#line 20911 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 255: /* TransactionStmt: ABORT_P opt_transaction */ #line 3 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_ROLLBACK; n->options = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20922 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 239: + case 256: /* TransactionStmt: BEGIN_P opt_transaction */ #line 10 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_BEGIN; (yyval.node) = (PGNode *)n; - ;} + } +#line 20932 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 240: + case 257: /* TransactionStmt: START opt_transaction */ #line 16 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_START; (yyval.node) = (PGNode *)n; - ;} + } +#line 20942 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 241: + case 258: /* TransactionStmt: COMMIT opt_transaction */ #line 22 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_COMMIT; n->options = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20953 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 242: + case 259: /* TransactionStmt: END_P opt_transaction */ #line 29 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_COMMIT; n->options = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20964 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 243: + case 260: /* TransactionStmt: ROLLBACK opt_transaction */ #line 36 "third_party/libpg_query/grammar/statements/transaction.y" - { + { PGTransactionStmt *n = makeNode(PGTransactionStmt); n->kind = PG_TRANS_STMT_ROLLBACK; n->options = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 20975 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 244: + case 261: /* opt_transaction: WORK */ #line 45 "third_party/libpg_query/grammar/statements/transaction.y" - {;} + {} +#line 20981 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 245: + case 262: /* opt_transaction: TRANSACTION */ #line 46 "third_party/libpg_query/grammar/statements/transaction.y" - {;} + {} +#line 20987 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 246: + case 263: /* opt_transaction: %empty */ #line 47 "third_party/libpg_query/grammar/statements/transaction.y" - {;} + {} +#line 20993 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 247: + case 264: /* UseStmt: USE_P qualified_name */ #line 3 "third_party/libpg_query/grammar/statements/use.y" - { + { PGUseStmt *n = makeNode(PGUseStmt); - n->name = (yyvsp[(2) - (2)].range); + n->name = (yyvsp[0].range); (yyval.node) = (PGNode *) n; - ;} + } +#line 21003 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 248: + case 265: /* CreateStmt: CREATE_P OptTemp TABLE qualified_name '(' OptTableElementList ')' OptWith OnCommitOption */ #line 9 "third_party/libpg_query/grammar/statements/create.y" - { + { PGCreateStmt *n = makeNode(PGCreateStmt); - (yyvsp[(4) - (9)].range)->relpersistence = (yyvsp[(2) - (9)].ival); - n->relation = (yyvsp[(4) - (9)].range); - n->tableElts = (yyvsp[(6) - (9)].list); + (yyvsp[-5].range)->relpersistence = (yyvsp[-7].ival); + n->relation = (yyvsp[-5].range); + n->tableElts = (yyvsp[-3].list); n->ofTypename = NULL; n->constraints = NIL; - n->options = (yyvsp[(8) - (9)].list); - n->oncommit = (yyvsp[(9) - (9)].oncommit); + n->options = (yyvsp[-1].list); + n->oncommit = (yyvsp[0].oncommit); n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 21020 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 249: + case 266: /* CreateStmt: CREATE_P OptTemp TABLE IF_P NOT EXISTS qualified_name '(' OptTableElementList ')' OptWith OnCommitOption */ #line 24 "third_party/libpg_query/grammar/statements/create.y" - { + { PGCreateStmt *n = makeNode(PGCreateStmt); - (yyvsp[(7) - (12)].range)->relpersistence = (yyvsp[(2) - (12)].ival); - n->relation = (yyvsp[(7) - (12)].range); - n->tableElts = (yyvsp[(9) - (12)].list); + (yyvsp[-5].range)->relpersistence = (yyvsp[-10].ival); + n->relation = (yyvsp[-5].range); + n->tableElts = (yyvsp[-3].list); n->ofTypename = NULL; n->constraints = NIL; - n->options = (yyvsp[(11) - (12)].list); - n->oncommit = (yyvsp[(12) - (12)].oncommit); + n->options = (yyvsp[-1].list); + n->oncommit = (yyvsp[0].oncommit); n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 21037 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 250: + case 267: /* CreateStmt: CREATE_P OR REPLACE OptTemp TABLE qualified_name '(' OptTableElementList ')' OptWith OnCommitOption */ #line 39 "third_party/libpg_query/grammar/statements/create.y" - { + { PGCreateStmt *n = makeNode(PGCreateStmt); - (yyvsp[(6) - (11)].range)->relpersistence = (yyvsp[(4) - (11)].ival); - n->relation = (yyvsp[(6) - (11)].range); - n->tableElts = (yyvsp[(8) - (11)].list); + (yyvsp[-5].range)->relpersistence = (yyvsp[-7].ival); + n->relation = (yyvsp[-5].range); + n->tableElts = (yyvsp[-3].list); n->ofTypename = NULL; n->constraints = NIL; - n->options = (yyvsp[(10) - (11)].list); - n->oncommit = (yyvsp[(11) - (11)].oncommit); + n->options = (yyvsp[-1].list); + n->oncommit = (yyvsp[0].oncommit); n->onconflict = PG_REPLACE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 21054 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 251: + case 268: /* ConstraintAttributeSpec: %empty */ #line 56 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 21060 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 252: + case 269: /* ConstraintAttributeSpec: ConstraintAttributeSpec ConstraintAttributeElem */ #line 58 "third_party/libpg_query/grammar/statements/create.y" - { + { /* * We must complain about conflicting options. * We could, but choose not to, complain about redundant * options (ie, where $2's bit is already set in $1). */ - int newspec = (yyvsp[(1) - (2)].ival) | (yyvsp[(2) - (2)].ival); + int newspec = (yyvsp[-1].ival) | (yyvsp[0].ival); /* special message for this case */ if ((newspec & (CAS_NOT_DEFERRABLE | CAS_INITIALLY_DEFERRED)) == (CAS_NOT_DEFERRABLE | CAS_INITIALLY_DEFERRED)) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("constraint declared INITIALLY DEFERRED must be DEFERRABLE"), - parser_errposition((yylsp[(2) - (2)])))); + parser_errposition((yylsp[0])))); /* generic message for other conflicts */ if ((newspec & (CAS_NOT_DEFERRABLE | CAS_DEFERRABLE)) == (CAS_NOT_DEFERRABLE | CAS_DEFERRABLE) || (newspec & (CAS_INITIALLY_IMMEDIATE | CAS_INITIALLY_DEFERRED)) == (CAS_INITIALLY_IMMEDIATE | CAS_INITIALLY_DEFERRED)) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("conflicting constraint properties"), - parser_errposition((yylsp[(2) - (2)])))); + parser_errposition((yylsp[0])))); (yyval.ival) = newspec; - ;} + } +#line 21088 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 253: + case 270: /* def_arg: func_type */ #line 84 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)(yyvsp[(1) - (1)].typnam); ;} + { (yyval.node) = (PGNode *)(yyvsp[0].typnam); } +#line 21094 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 254: + case 271: /* def_arg: reserved_keyword */ #line 85 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)makeString(pstrdup((yyvsp[(1) - (1)].keyword))); ;} + { (yyval.node) = (PGNode *)makeString(pstrdup((yyvsp[0].keyword))); } +#line 21100 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 255: + case 272: /* def_arg: qual_all_Op */ #line 86 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)(yyvsp[(1) - (1)].list); ;} + { (yyval.node) = (PGNode *)(yyvsp[0].list); } +#line 21106 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 256: + case 273: /* def_arg: NumericOnly */ #line 87 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)(yyvsp[(1) - (1)].value); ;} + { (yyval.node) = (PGNode *)(yyvsp[0].value); } +#line 21112 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 257: + case 274: /* def_arg: Sconst */ #line 88 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)makeString((yyvsp[(1) - (1)].str)); ;} + { (yyval.node) = (PGNode *)makeString((yyvsp[0].str)); } +#line 21118 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 258: + case 275: /* def_arg: NONE */ #line 89 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *)makeString(pstrdup((yyvsp[(1) - (1)].keyword))); ;} + { (yyval.node) = (PGNode *)makeString(pstrdup((yyvsp[0].keyword))); } +#line 21124 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 259: + case 276: /* OptParenthesizedSeqOptList: '(' SeqOptList ')' */ #line 93 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21130 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 260: + case 277: /* OptParenthesizedSeqOptList: %empty */ #line 94 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21136 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 261: + case 278: /* generic_option_arg: Sconst */ #line 99 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); ;} + { (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); } +#line 21142 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 262: + case 279: /* key_action: NO ACTION */ #line 104 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_FKCONSTR_ACTION_NOACTION; ;} + { (yyval.ival) = PG_FKCONSTR_ACTION_NOACTION; } +#line 21148 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 263: + case 280: /* key_action: RESTRICT */ #line 105 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_FKCONSTR_ACTION_RESTRICT; ;} + { (yyval.ival) = PG_FKCONSTR_ACTION_RESTRICT; } +#line 21154 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 264: + case 281: /* key_action: CASCADE */ #line 106 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_FKCONSTR_ACTION_CASCADE; ;} + { (yyval.ival) = PG_FKCONSTR_ACTION_CASCADE; } +#line 21160 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 265: + case 282: /* key_action: SET NULL_P */ #line 107 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_FKCONSTR_ACTION_SETNULL; ;} + { (yyval.ival) = PG_FKCONSTR_ACTION_SETNULL; } +#line 21166 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 266: + case 283: /* key_action: SET DEFAULT */ #line 108 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_FKCONSTR_ACTION_SETDEFAULT; ;} + { (yyval.ival) = PG_FKCONSTR_ACTION_SETDEFAULT; } +#line 21172 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 267: + case 284: /* ColConstraint: CONSTRAINT name ColConstraintElem */ #line 114 "third_party/libpg_query/grammar/statements/create.y" - { - PGConstraint *n = castNode(PGConstraint, (yyvsp[(3) - (3)].node)); - n->conname = (yyvsp[(2) - (3)].str); - n->location = (yylsp[(1) - (3)]); + { + PGConstraint *n = castNode(PGConstraint, (yyvsp[0].node)); + n->conname = (yyvsp[-1].str); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *) n; - ;} + } +#line 21183 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 268: + case 285: /* ColConstraint: ColConstraintElem */ #line 120 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21189 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 269: + case 286: /* ColConstraint: ConstraintAttr */ #line 121 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21195 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 270: + case 287: /* ColConstraint: COLLATE any_name */ #line 123 "third_party/libpg_query/grammar/statements/create.y" - { + { /* * Note: the PGCollateClause is momentarily included in * the list built by ColQualList, but we split it out @@ -21207,155 +21204,169 @@ YYLTYPE yylloc; */ PGCollateClause *n = makeNode(PGCollateClause); n->arg = NULL; - n->collname = (yyvsp[(2) - (2)].list); - n->location = (yylsp[(1) - (2)]); + n->collname = (yyvsp[0].list); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *) n; - ;} + } +#line 21212 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 271: + case 288: /* ColConstraintElem: NOT NULL_P */ #line 140 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_NOTNULL; - n->location = (yylsp[(1) - (2)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21223 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 272: + case 289: /* ColConstraintElem: NULL_P */ #line 147 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_NULL; - n->location = (yylsp[(1) - (1)]); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21234 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 273: + case 290: /* ColConstraintElem: UNIQUE opt_definition */ #line 154 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_UNIQUE; - n->location = (yylsp[(1) - (2)]); + n->location = (yylsp[-1]); n->keys = NULL; - n->options = (yyvsp[(2) - (2)].list); + n->options = (yyvsp[0].list); n->indexname = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 21248 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 274: + case 291: /* ColConstraintElem: PRIMARY KEY opt_definition */ #line 164 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_PRIMARY; - n->location = (yylsp[(1) - (3)]); + n->location = (yylsp[-2]); n->keys = NULL; - n->options = (yyvsp[(3) - (3)].list); + n->options = (yyvsp[0].list); n->indexname = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 21262 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 275: + case 292: /* ColConstraintElem: CHECK_P '(' a_expr ')' opt_no_inherit */ #line 174 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_CHECK; - n->location = (yylsp[(1) - (5)]); - n->is_no_inherit = (yyvsp[(5) - (5)].boolean); - n->raw_expr = (yyvsp[(3) - (5)].node); + n->location = (yylsp[-4]); + n->is_no_inherit = (yyvsp[0].boolean); + n->raw_expr = (yyvsp[-2].node); n->cooked_expr = NULL; n->skip_validation = false; n->initially_valid = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 21278 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 276: + case 293: /* ColConstraintElem: USING COMPRESSION name */ #line 186 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_COMPRESSION; - n->location = (yylsp[(1) - (3)]); - n->compression_name = (yyvsp[(3) - (3)].str); + n->location = (yylsp[-2]); + n->compression_name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 21290 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 277: + case 294: /* ColConstraintElem: DEFAULT b_expr */ #line 194 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_DEFAULT; - n->location = (yylsp[(1) - (2)]); - n->raw_expr = (yyvsp[(2) - (2)].node); + n->location = (yylsp[-1]); + n->raw_expr = (yyvsp[0].node); n->cooked_expr = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 21303 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 278: + case 295: /* ColConstraintElem: REFERENCES qualified_name opt_column_list key_match key_actions */ #line 203 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_FOREIGN; - n->location = (yylsp[(1) - (5)]); - n->pktable = (yyvsp[(2) - (5)].range); + n->location = (yylsp[-4]); + n->pktable = (yyvsp[-3].range); n->fk_attrs = NIL; - n->pk_attrs = (yyvsp[(3) - (5)].list); - n->fk_matchtype = (yyvsp[(4) - (5)].ival); - n->fk_upd_action = (char) ((yyvsp[(5) - (5)].ival) >> 8); - n->fk_del_action = (char) ((yyvsp[(5) - (5)].ival) & 0xFF); + n->pk_attrs = (yyvsp[-2].list); + n->fk_matchtype = (yyvsp[-1].ival); + n->fk_upd_action = (char) ((yyvsp[0].ival) >> 8); + n->fk_del_action = (char) ((yyvsp[0].ival) & 0xFF); n->skip_validation = false; n->initially_valid = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 21322 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 279: + case 296: /* GeneratedColumnType: VIRTUAL */ #line 220 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.constr) = PG_CONSTR_GENERATED_VIRTUAL; ;} + { (yyval.constr) = PG_CONSTR_GENERATED_VIRTUAL; } +#line 21328 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 280: + case 297: /* GeneratedColumnType: STORED */ #line 221 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.constr) = PG_CONSTR_GENERATED_STORED; ;} + { (yyval.constr) = PG_CONSTR_GENERATED_STORED; } +#line 21334 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 281: + case 298: /* opt_GeneratedColumnType: GeneratedColumnType */ #line 225 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.constr) = (yyvsp[(1) - (1)].constr); ;} + { (yyval.constr) = (yyvsp[0].constr); } +#line 21340 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 282: + case 299: /* opt_GeneratedColumnType: %empty */ #line 226 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.constr) = PG_CONSTR_GENERATED_VIRTUAL; ;} + { (yyval.constr) = PG_CONSTR_GENERATED_VIRTUAL; } +#line 21346 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 283: + case 300: /* GeneratedConstraintElem: GENERATED generated_when AS IDENTITY_P OptParenthesizedSeqOptList */ #line 231 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_IDENTITY; - n->generated_when = (yyvsp[(2) - (5)].ival); - n->options = (yyvsp[(5) - (5)].list); - n->location = (yylsp[(1) - (5)]); + n->generated_when = (yyvsp[-3].ival); + n->options = (yyvsp[0].list); + n->location = (yylsp[-4]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21359 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 284: + case 301: /* GeneratedConstraintElem: GENERATED generated_when AS '(' a_expr ')' opt_GeneratedColumnType */ #line 240 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); - n->contype = (yyvsp[(7) - (7)].constr); - n->generated_when = (yyvsp[(2) - (7)].ival); - n->raw_expr = (yyvsp[(5) - (7)].node); + n->contype = (yyvsp[0].constr); + n->generated_when = (yyvsp[-5].ival); + n->raw_expr = (yyvsp[-2].node); n->cooked_expr = NULL; - n->location = (yylsp[(1) - (7)]); + n->location = (yylsp[-6]); /* * Can't do this in the grammar because of shift/reduce @@ -21363,293 +21374,341 @@ YYLTYPE yylloc; * DEFAULT, but generated columns only allow ALWAYS.) We * can also give a more useful error message and location. */ - if ((yyvsp[(2) - (7)].ival) != PG_ATTRIBUTE_IDENTITY_ALWAYS) + if ((yyvsp[-5].ival) != PG_ATTRIBUTE_IDENTITY_ALWAYS) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("for a generated column, GENERATED ALWAYS must be specified"), - parser_errposition((yylsp[(2) - (7)])))); + parser_errposition((yylsp[-5])))); (yyval.node) = (PGNode *)n; - ;} + } +#line 21386 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 285: + case 302: /* GeneratedConstraintElem: AS '(' a_expr ')' opt_GeneratedColumnType */ #line 263 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); - n->contype = (yyvsp[(5) - (5)].constr); + n->contype = (yyvsp[0].constr); n->generated_when = PG_ATTRIBUTE_IDENTITY_ALWAYS; - n->raw_expr = (yyvsp[(3) - (5)].node); + n->raw_expr = (yyvsp[-2].node); n->cooked_expr = NULL; - n->location = (yylsp[(1) - (5)]); + n->location = (yylsp[-4]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21400 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 286: + case 303: /* generic_option_elem: generic_option_name generic_option_arg */ #line 277 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[-1].str), (yyvsp[0].node), (yylsp[-1])); + } +#line 21408 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 287: + case 304: /* key_update: ON UPDATE key_action */ #line 283 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (yyvsp[(3) - (3)].ival); ;} + { (yyval.ival) = (yyvsp[0].ival); } +#line 21414 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 288: + case 305: /* key_actions: key_update */ #line 289 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = ((yyvsp[(1) - (1)].ival) << 8) | (PG_FKCONSTR_ACTION_NOACTION & 0xFF); ;} + { (yyval.ival) = ((yyvsp[0].ival) << 8) | (PG_FKCONSTR_ACTION_NOACTION & 0xFF); } +#line 21420 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 289: + case 306: /* key_actions: key_delete */ #line 291 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (PG_FKCONSTR_ACTION_NOACTION << 8) | ((yyvsp[(1) - (1)].ival) & 0xFF); ;} + { (yyval.ival) = (PG_FKCONSTR_ACTION_NOACTION << 8) | ((yyvsp[0].ival) & 0xFF); } +#line 21426 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 290: + case 307: /* key_actions: key_update key_delete */ #line 293 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = ((yyvsp[(1) - (2)].ival) << 8) | ((yyvsp[(2) - (2)].ival) & 0xFF); ;} + { (yyval.ival) = ((yyvsp[-1].ival) << 8) | ((yyvsp[0].ival) & 0xFF); } +#line 21432 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 291: + case 308: /* key_actions: key_delete key_update */ #line 295 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = ((yyvsp[(2) - (2)].ival) << 8) | ((yyvsp[(1) - (2)].ival) & 0xFF); ;} + { (yyval.ival) = ((yyvsp[0].ival) << 8) | ((yyvsp[-1].ival) & 0xFF); } +#line 21438 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 292: + case 309: /* key_actions: %empty */ #line 297 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (PG_FKCONSTR_ACTION_NOACTION << 8) | (PG_FKCONSTR_ACTION_NOACTION & 0xFF); ;} + { (yyval.ival) = (PG_FKCONSTR_ACTION_NOACTION << 8) | (PG_FKCONSTR_ACTION_NOACTION & 0xFF); } +#line 21444 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 293: + case 310: /* OnCommitOption: ON COMMIT DROP */ #line 300 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.oncommit) = ONCOMMIT_DROP; ;} + { (yyval.oncommit) = ONCOMMIT_DROP; } +#line 21450 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 294: + case 311: /* OnCommitOption: ON COMMIT DELETE_P ROWS */ #line 301 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.oncommit) = PG_ONCOMMIT_DELETE_ROWS; ;} + { (yyval.oncommit) = PG_ONCOMMIT_DELETE_ROWS; } +#line 21456 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 295: + case 312: /* OnCommitOption: ON COMMIT PRESERVE ROWS */ #line 302 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.oncommit) = PG_ONCOMMIT_PRESERVE_ROWS; ;} + { (yyval.oncommit) = PG_ONCOMMIT_PRESERVE_ROWS; } +#line 21462 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 296: + case 313: /* OnCommitOption: %empty */ #line 303 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.oncommit) = PG_ONCOMMIT_NOOP; ;} + { (yyval.oncommit) = PG_ONCOMMIT_NOOP; } +#line 21468 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 297: + case 314: /* reloptions: '(' reloption_list ')' */ #line 308 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21474 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 298: + case 315: /* opt_no_inherit: NO INHERIT */ #line 312 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 21480 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 299: + case 316: /* opt_no_inherit: %empty */ #line 313 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 21486 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 300: + case 317: /* TableConstraint: CONSTRAINT name ConstraintElem */ #line 319 "third_party/libpg_query/grammar/statements/create.y" - { - PGConstraint *n = castNode(PGConstraint, (yyvsp[(3) - (3)].node)); - n->conname = (yyvsp[(2) - (3)].str); - n->location = (yylsp[(1) - (3)]); + { + PGConstraint *n = castNode(PGConstraint, (yyvsp[0].node)); + n->conname = (yyvsp[-1].str); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *) n; - ;} + } +#line 21497 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 301: + case 318: /* TableConstraint: ConstraintElem */ #line 325 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21503 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 302: + case 319: /* TableLikeOption: COMMENTS */ #line 330 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_COMMENTS; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_COMMENTS; } +#line 21509 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 303: + case 320: /* TableLikeOption: CONSTRAINTS */ #line 331 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_CONSTRAINTS; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_CONSTRAINTS; } +#line 21515 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 304: + case 321: /* TableLikeOption: DEFAULTS */ #line 332 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_DEFAULTS; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_DEFAULTS; } +#line 21521 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 305: + case 322: /* TableLikeOption: IDENTITY_P */ #line 333 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_IDENTITY; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_IDENTITY; } +#line 21527 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 306: + case 323: /* TableLikeOption: INDEXES */ #line 334 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_INDEXES; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_INDEXES; } +#line 21533 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 307: + case 324: /* TableLikeOption: STATISTICS */ #line 335 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_STATISTICS; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_STATISTICS; } +#line 21539 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 308: + case 325: /* TableLikeOption: STORAGE */ #line 336 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_STORAGE; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_STORAGE; } +#line 21545 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 309: + case 326: /* TableLikeOption: ALL */ #line 337 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_CREATE_TABLE_LIKE_ALL; ;} + { (yyval.ival) = PG_CREATE_TABLE_LIKE_ALL; } +#line 21551 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 310: + case 327: /* reloption_list: reloption_elem */ #line 343 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); ;} + { (yyval.list) = list_make1((yyvsp[0].defelt)); } +#line 21557 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 311: + case 328: /* reloption_list: reloption_list ',' reloption_elem */ #line 344 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].defelt)); } +#line 21563 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 312: + case 329: /* ExistingIndex: USING INDEX index_name */ #line 348 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.str) = (yyvsp[(3) - (3)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 21569 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 313: + case 330: /* ConstraintAttr: DEFERRABLE */ #line 354 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_ATTR_DEFERRABLE; - n->location = (yylsp[(1) - (1)]); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21580 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 314: + case 331: /* ConstraintAttr: NOT DEFERRABLE */ #line 361 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_ATTR_NOT_DEFERRABLE; - n->location = (yylsp[(1) - (2)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21591 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 315: + case 332: /* ConstraintAttr: INITIALLY DEFERRED */ #line 368 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_ATTR_DEFERRED; - n->location = (yylsp[(1) - (2)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21602 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 316: + case 333: /* ConstraintAttr: INITIALLY IMMEDIATE */ #line 375 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_ATTR_IMMEDIATE; - n->location = (yylsp[(1) - (2)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21613 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 317: + case 334: /* OptWith: WITH reloptions */ #line 386 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 21619 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 318: + case 335: /* OptWith: WITH OIDS */ #line 387 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = list_make1(makeDefElem("oids", (PGNode *) makeInteger(true), (yylsp[(1) - (2)]))); ;} + { (yyval.list) = list_make1(makeDefElem("oids", (PGNode *) makeInteger(true), (yylsp[-1]))); } +#line 21625 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 319: + case 336: /* OptWith: WITHOUT OIDS */ #line 388 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = list_make1(makeDefElem("oids", (PGNode *) makeInteger(false), (yylsp[(1) - (2)]))); ;} + { (yyval.list) = list_make1(makeDefElem("oids", (PGNode *) makeInteger(false), (yylsp[-1]))); } +#line 21631 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 320: + case 337: /* OptWith: %empty */ #line 389 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21637 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 321: + case 338: /* definition: '(' def_list ')' */ #line 393 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21643 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 322: + case 339: /* TableLikeOptionList: TableLikeOptionList INCLUDING TableLikeOption */ #line 398 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (yyvsp[(1) - (3)].ival) | (yyvsp[(3) - (3)].ival); ;} + { (yyval.ival) = (yyvsp[-2].ival) | (yyvsp[0].ival); } +#line 21649 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 323: + case 340: /* TableLikeOptionList: TableLikeOptionList EXCLUDING TableLikeOption */ #line 399 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (yyvsp[(1) - (3)].ival) & ~(yyvsp[(3) - (3)].ival); ;} + { (yyval.ival) = (yyvsp[-2].ival) & ~(yyvsp[0].ival); } +#line 21655 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 324: + case 341: /* TableLikeOptionList: %empty */ #line 400 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 21661 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 325: + case 342: /* generic_option_name: ColLabel */ #line 405 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 21667 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 326: + case 343: /* ConstraintAttributeElem: NOT DEFERRABLE */ #line 410 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_NOT_DEFERRABLE; ;} + { (yyval.ival) = CAS_NOT_DEFERRABLE; } +#line 21673 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 327: + case 344: /* ConstraintAttributeElem: DEFERRABLE */ #line 411 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_DEFERRABLE; ;} + { (yyval.ival) = CAS_DEFERRABLE; } +#line 21679 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 328: + case 345: /* ConstraintAttributeElem: INITIALLY IMMEDIATE */ #line 412 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_INITIALLY_IMMEDIATE; ;} + { (yyval.ival) = CAS_INITIALLY_IMMEDIATE; } +#line 21685 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 329: + case 346: /* ConstraintAttributeElem: INITIALLY DEFERRED */ #line 413 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_INITIALLY_DEFERRED; ;} + { (yyval.ival) = CAS_INITIALLY_DEFERRED; } +#line 21691 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 330: + case 347: /* ConstraintAttributeElem: NOT VALID */ #line 414 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_NOT_VALID; ;} + { (yyval.ival) = CAS_NOT_VALID; } +#line 21697 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 331: + case 348: /* ConstraintAttributeElem: NO INHERIT */ #line 415 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = CAS_NO_INHERIT; ;} + { (yyval.ival) = CAS_NO_INHERIT; } +#line 21703 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 332: + case 349: /* columnDef: ColId Typename ColQualList */ #line 421 "third_party/libpg_query/grammar/statements/create.y" - { + { PGColumnDef *n = makeNode(PGColumnDef); n->category = COL_STANDARD; - n->colname = (yyvsp[(1) - (3)].str); - n->typeName = (yyvsp[(2) - (3)].typnam); + n->colname = (yyvsp[-2].str); + n->typeName = (yyvsp[-1].typnam); n->inhcount = 0; n->is_local = true; n->is_not_null = false; @@ -21658,20 +21717,21 @@ YYLTYPE yylloc; n->raw_default = NULL; n->cooked_default = NULL; n->collOid = InvalidOid; - SplitColQualList((yyvsp[(3) - (3)].list), &n->constraints, &n->collClause, + SplitColQualList((yyvsp[0].list), &n->constraints, &n->collClause, yyscanner); - n->location = (yylsp[(1) - (3)]); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21726 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 333: + case 350: /* columnDef: ColId opt_Typename GeneratedConstraintElem ColQualList */ #line 441 "third_party/libpg_query/grammar/statements/create.y" - { + { PGColumnDef *n = makeNode(PGColumnDef); n->category = COL_GENERATED; - n->colname = (yyvsp[(1) - (4)].str); - n->typeName = (yyvsp[(2) - (4)].typnam); + n->colname = (yyvsp[-3].str); + n->typeName = (yyvsp[-2].typnam); n->inhcount = 0; n->is_local = true; n->is_not_null = false; @@ -21681,1330 +21741,1525 @@ YYLTYPE yylloc; n->cooked_default = NULL; n->collOid = InvalidOid; // merge the constraints with the generated column constraint - auto constraints = (yyvsp[(4) - (4)].list); + auto constraints = (yyvsp[0].list); if (constraints) { - constraints = lappend(constraints, (yyvsp[(3) - (4)].node)); + constraints = lappend(constraints, (yyvsp[-1].node)); } else { - constraints = list_make1((yyvsp[(3) - (4)].node)); + constraints = list_make1((yyvsp[-1].node)); } SplitColQualList(constraints, &n->constraints, &n->collClause, yyscanner); - n->location = (yylsp[(1) - (4)]); + n->location = (yylsp[-3]); (yyval.node) = (PGNode *)n; - ;} + } +#line 21756 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 334: + case 351: /* def_list: def_elem */ #line 469 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); ;} + { (yyval.list) = list_make1((yyvsp[0].defelt)); } +#line 21762 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 335: + case 352: /* def_list: def_list ',' def_elem */ #line 470 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].defelt)); } +#line 21768 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 336: + case 353: /* index_name: ColId */ #line 474 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 21774 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 337: + case 354: /* TableElement: columnDef */ #line 478 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21780 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 338: + case 355: /* TableElement: TableLikeClause */ #line 479 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21786 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 339: + case 356: /* TableElement: TableConstraint */ #line 480 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 21792 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 340: + case 357: /* def_elem: ColLabel '=' def_arg */ #line 485 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (3)].str), (PGNode *) (yyvsp[(3) - (3)].node), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[-2].str), (PGNode *) (yyvsp[0].node), (yylsp[-2])); + } +#line 21800 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 341: + case 358: /* def_elem: ColLabel */ #line 489 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (1)].str), NULL, (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[0].str), NULL, (yylsp[0])); + } +#line 21808 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 342: + case 359: /* opt_definition: WITH definition */ #line 496 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 21814 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 343: + case 360: /* opt_definition: %empty */ #line 497 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21820 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 344: + case 361: /* OptTableElementList: TableElementList */ #line 502 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 21826 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 345: + case 362: /* OptTableElementList: TableElementList ',' */ #line 503 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21832 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 346: + case 363: /* OptTableElementList: %empty */ #line 504 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21838 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 347: + case 364: /* columnElem: ColId */ #line 509 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); - ;} + { + (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); + } +#line 21846 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 348: + case 365: /* opt_column_list: '(' columnList ')' */ #line 516 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21852 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 349: + case 366: /* opt_column_list: %empty */ #line 517 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21858 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 350: + case 367: /* ColQualList: ColQualList ColConstraint */ #line 522 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 21864 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 351: + case 368: /* ColQualList: %empty */ #line 523 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 21870 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 352: + case 369: /* key_delete: ON DELETE_P key_action */ #line 527 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = (yyvsp[(3) - (3)].ival); ;} + { (yyval.ival) = (yyvsp[0].ival); } +#line 21876 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 353: + case 370: /* reloption_elem: ColLabel '=' def_arg */ #line 533 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (3)].str), (PGNode *) (yyvsp[(3) - (3)].node), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[-2].str), (PGNode *) (yyvsp[0].node), (yylsp[-2])); + } +#line 21884 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 354: + case 371: /* reloption_elem: ColLabel */ #line 537 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (1)].str), NULL, (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[0].str), NULL, (yylsp[0])); + } +#line 21892 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 355: + case 372: /* reloption_elem: ColLabel '.' ColLabel '=' def_arg */ #line 541 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElemExtended((yyvsp[(1) - (5)].str), (yyvsp[(3) - (5)].str), (PGNode *) (yyvsp[(5) - (5)].node), - PG_DEFELEM_UNSPEC, (yylsp[(1) - (5)])); - ;} + { + (yyval.defelt) = makeDefElemExtended((yyvsp[-4].str), (yyvsp[-2].str), (PGNode *) (yyvsp[0].node), + PG_DEFELEM_UNSPEC, (yylsp[-4])); + } +#line 21901 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 356: + case 373: /* reloption_elem: ColLabel '.' ColLabel */ #line 546 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.defelt) = makeDefElemExtended((yyvsp[(1) - (3)].str), (yyvsp[(3) - (3)].str), NULL, PG_DEFELEM_UNSPEC, (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElemExtended((yyvsp[-2].str), (yyvsp[0].str), NULL, PG_DEFELEM_UNSPEC, (yylsp[-2])); + } +#line 21909 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 357: + case 374: /* columnList: columnElem */ #line 553 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 21915 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 358: + case 375: /* columnList: columnList ',' columnElem */ #line 554 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 21921 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 359: + case 376: /* columnList_opt_comma: columnList */ #line 558 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 21927 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 360: + case 377: /* columnList_opt_comma: columnList ',' */ #line 559 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 21933 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 361: + case 378: /* func_type: Typename */ #line 563 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 21939 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 362: + case 379: /* func_type: type_function_name attrs '%' TYPE_P */ #line 565 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.typnam) = makeTypeNameFromNameList(lcons(makeString((yyvsp[(1) - (4)].str)), (yyvsp[(2) - (4)].list))); + { + (yyval.typnam) = makeTypeNameFromNameList(lcons(makeString((yyvsp[-3].str)), (yyvsp[-2].list))); (yyval.typnam)->pct_type = true; - (yyval.typnam)->location = (yylsp[(1) - (4)]); - ;} + (yyval.typnam)->location = (yylsp[-3]); + } +#line 21949 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 363: + case 380: /* func_type: SETOF type_function_name attrs '%' TYPE_P */ #line 571 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.typnam) = makeTypeNameFromNameList(lcons(makeString((yyvsp[(2) - (5)].str)), (yyvsp[(3) - (5)].list))); + { + (yyval.typnam) = makeTypeNameFromNameList(lcons(makeString((yyvsp[-3].str)), (yyvsp[-2].list))); (yyval.typnam)->pct_type = true; (yyval.typnam)->setof = true; - (yyval.typnam)->location = (yylsp[(2) - (5)]); - ;} + (yyval.typnam)->location = (yylsp[-3]); + } +#line 21960 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 364: + case 381: /* ConstraintElem: CHECK_P '(' a_expr ')' ConstraintAttributeSpec */ #line 582 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_CHECK; - n->location = (yylsp[(1) - (5)]); - n->raw_expr = (yyvsp[(3) - (5)].node); + n->location = (yylsp[-4]); + n->raw_expr = (yyvsp[-2].node); n->cooked_expr = NULL; - processCASbits((yyvsp[(5) - (5)].ival), (yylsp[(5) - (5)]), "CHECK", + processCASbits((yyvsp[0].ival), (yylsp[0]), "CHECK", NULL, NULL, &n->skip_validation, &n->is_no_inherit, yyscanner); n->initially_valid = !n->skip_validation; (yyval.node) = (PGNode *)n; - ;} + } +#line 21977 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 365: + case 382: /* ConstraintElem: UNIQUE '(' columnList_opt_comma ')' opt_definition ConstraintAttributeSpec */ #line 596 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_UNIQUE; - n->location = (yylsp[(1) - (6)]); - n->keys = (yyvsp[(3) - (6)].list); - n->options = (yyvsp[(5) - (6)].list); + n->location = (yylsp[-5]); + n->keys = (yyvsp[-3].list); + n->options = (yyvsp[-1].list); n->indexname = NULL; - processCASbits((yyvsp[(6) - (6)].ival), (yylsp[(6) - (6)]), "UNIQUE", + processCASbits((yyvsp[0].ival), (yylsp[0]), "UNIQUE", &n->deferrable, &n->initdeferred, NULL, NULL, yyscanner); (yyval.node) = (PGNode *)n; - ;} + } +#line 21994 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 366: + case 383: /* ConstraintElem: UNIQUE ExistingIndex ConstraintAttributeSpec */ #line 609 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_UNIQUE; - n->location = (yylsp[(1) - (3)]); + n->location = (yylsp[-2]); n->keys = NIL; n->options = NIL; - n->indexname = (yyvsp[(2) - (3)].str); + n->indexname = (yyvsp[-1].str); n->indexspace = NULL; - processCASbits((yyvsp[(3) - (3)].ival), (yylsp[(3) - (3)]), "UNIQUE", + processCASbits((yyvsp[0].ival), (yylsp[0]), "UNIQUE", &n->deferrable, &n->initdeferred, NULL, NULL, yyscanner); (yyval.node) = (PGNode *)n; - ;} + } +#line 22012 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 367: + case 384: /* ConstraintElem: PRIMARY KEY '(' columnList_opt_comma ')' opt_definition ConstraintAttributeSpec */ #line 624 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_PRIMARY; - n->location = (yylsp[(1) - (7)]); - n->keys = (yyvsp[(4) - (7)].list); - n->options = (yyvsp[(6) - (7)].list); + n->location = (yylsp[-6]); + n->keys = (yyvsp[-3].list); + n->options = (yyvsp[-1].list); n->indexname = NULL; - processCASbits((yyvsp[(7) - (7)].ival), (yylsp[(7) - (7)]), "PRIMARY KEY", + processCASbits((yyvsp[0].ival), (yylsp[0]), "PRIMARY KEY", &n->deferrable, &n->initdeferred, NULL, NULL, yyscanner); (yyval.node) = (PGNode *)n; - ;} + } +#line 22029 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 368: + case 385: /* ConstraintElem: PRIMARY KEY ExistingIndex ConstraintAttributeSpec */ #line 637 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_PRIMARY; - n->location = (yylsp[(1) - (4)]); + n->location = (yylsp[-3]); n->keys = NIL; n->options = NIL; - n->indexname = (yyvsp[(3) - (4)].str); + n->indexname = (yyvsp[-1].str); n->indexspace = NULL; - processCASbits((yyvsp[(4) - (4)].ival), (yylsp[(4) - (4)]), "PRIMARY KEY", + processCASbits((yyvsp[0].ival), (yylsp[0]), "PRIMARY KEY", &n->deferrable, &n->initdeferred, NULL, NULL, yyscanner); (yyval.node) = (PGNode *)n; - ;} + } +#line 22047 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 369: + case 386: /* ConstraintElem: FOREIGN KEY '(' columnList_opt_comma ')' REFERENCES qualified_name opt_column_list key_match key_actions ConstraintAttributeSpec */ #line 652 "third_party/libpg_query/grammar/statements/create.y" - { + { PGConstraint *n = makeNode(PGConstraint); n->contype = PG_CONSTR_FOREIGN; - n->location = (yylsp[(1) - (11)]); - n->pktable = (yyvsp[(7) - (11)].range); - n->fk_attrs = (yyvsp[(4) - (11)].list); - n->pk_attrs = (yyvsp[(8) - (11)].list); - n->fk_matchtype = (yyvsp[(9) - (11)].ival); - n->fk_upd_action = (char) ((yyvsp[(10) - (11)].ival) >> 8); - n->fk_del_action = (char) ((yyvsp[(10) - (11)].ival) & 0xFF); - processCASbits((yyvsp[(11) - (11)].ival), (yylsp[(11) - (11)]), "FOREIGN KEY", + n->location = (yylsp[-10]); + n->pktable = (yyvsp[-4].range); + n->fk_attrs = (yyvsp[-7].list); + n->pk_attrs = (yyvsp[-3].list); + n->fk_matchtype = (yyvsp[-2].ival); + n->fk_upd_action = (char) ((yyvsp[-1].ival) >> 8); + n->fk_del_action = (char) ((yyvsp[-1].ival) & 0xFF); + processCASbits((yyvsp[0].ival), (yylsp[0]), "FOREIGN KEY", &n->deferrable, &n->initdeferred, &n->skip_validation, NULL, yyscanner); n->initially_valid = !n->skip_validation; (yyval.node) = (PGNode *)n; - ;} + } +#line 22069 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 370: + case 387: /* TableElementList: TableElement */ #line 674 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 22077 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 371: + case 388: /* TableElementList: TableElementList ',' TableElement */ #line 678 "third_party/libpg_query/grammar/statements/create.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 22085 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 372: + case 389: /* key_match: MATCH FULL */ #line 685 "third_party/libpg_query/grammar/statements/create.y" - { + { (yyval.ival) = PG_FKCONSTR_MATCH_FULL; - ;} + } +#line 22093 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 373: + case 390: /* key_match: MATCH PARTIAL */ #line 689 "third_party/libpg_query/grammar/statements/create.y" - { + { ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("MATCH PARTIAL not yet implemented"), - parser_errposition((yylsp[(1) - (2)])))); + parser_errposition((yylsp[-1])))); (yyval.ival) = PG_FKCONSTR_MATCH_PARTIAL; - ;} + } +#line 22105 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 374: + case 391: /* key_match: MATCH SIMPLE */ #line 697 "third_party/libpg_query/grammar/statements/create.y" - { + { (yyval.ival) = PG_FKCONSTR_MATCH_SIMPLE; - ;} + } +#line 22113 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 375: + case 392: /* key_match: %empty */ #line 701 "third_party/libpg_query/grammar/statements/create.y" - { + { (yyval.ival) = PG_FKCONSTR_MATCH_SIMPLE; - ;} + } +#line 22121 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 376: + case 393: /* TableLikeClause: LIKE qualified_name TableLikeOptionList */ #line 709 "third_party/libpg_query/grammar/statements/create.y" - { + { PGTableLikeClause *n = makeNode(PGTableLikeClause); - n->relation = (yyvsp[(2) - (3)].range); - n->options = (yyvsp[(3) - (3)].ival); + n->relation = (yyvsp[-1].range); + n->options = (yyvsp[0].ival); (yyval.node) = (PGNode *)n; - ;} + } +#line 22132 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 377: + case 394: /* OptTemp: TEMPORARY */ #line 718 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_RELPERSISTENCE_TEMP; ;} + { (yyval.ival) = PG_RELPERSISTENCE_TEMP; } +#line 22138 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 378: + case 395: /* OptTemp: TEMP */ #line 719 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_RELPERSISTENCE_TEMP; ;} + { (yyval.ival) = PG_RELPERSISTENCE_TEMP; } +#line 22144 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 379: + case 396: /* OptTemp: LOCAL TEMPORARY */ #line 720 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_RELPERSISTENCE_TEMP; ;} + { (yyval.ival) = PG_RELPERSISTENCE_TEMP; } +#line 22150 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 380: + case 397: /* OptTemp: LOCAL TEMP */ #line 721 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_RELPERSISTENCE_TEMP; ;} + { (yyval.ival) = PG_RELPERSISTENCE_TEMP; } +#line 22156 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 381: + case 398: /* OptTemp: GLOBAL TEMPORARY */ #line 723 "third_party/libpg_query/grammar/statements/create.y" - { + { ereport(PGWARNING, (errmsg("GLOBAL is deprecated in temporary table creation"), - parser_errposition((yylsp[(1) - (2)])))); + parser_errposition((yylsp[-1])))); (yyval.ival) = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 22167 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 382: + case 399: /* OptTemp: GLOBAL TEMP */ #line 730 "third_party/libpg_query/grammar/statements/create.y" - { + { ereport(PGWARNING, (errmsg("GLOBAL is deprecated in temporary table creation"), - parser_errposition((yylsp[(1) - (2)])))); + parser_errposition((yylsp[-1])))); (yyval.ival) = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 22178 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 383: + case 400: /* OptTemp: UNLOGGED */ #line 736 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_RELPERSISTENCE_UNLOGGED; ;} + { (yyval.ival) = PG_RELPERSISTENCE_UNLOGGED; } +#line 22184 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 384: + case 401: /* OptTemp: %empty */ #line 737 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = RELPERSISTENCE_PERMANENT; ;} + { (yyval.ival) = RELPERSISTENCE_PERMANENT; } +#line 22190 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 385: + case 402: /* generated_when: ALWAYS */ #line 742 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = PG_ATTRIBUTE_IDENTITY_ALWAYS; ;} + { (yyval.ival) = PG_ATTRIBUTE_IDENTITY_ALWAYS; } +#line 22196 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 386: + case 403: /* generated_when: BY DEFAULT */ #line 743 "third_party/libpg_query/grammar/statements/create.y" - { (yyval.ival) = ATTRIBUTE_IDENTITY_BY_DEFAULT; ;} + { (yyval.ival) = ATTRIBUTE_IDENTITY_BY_DEFAULT; } +#line 22202 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 387: + case 404: /* DropStmt: DROP drop_type_any_name IF_P EXISTS any_name_list opt_drop_behavior */ #line 10 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (6)].objtype); + n->removeType = (yyvsp[-4].objtype); n->missing_ok = true; - n->objects = (yyvsp[(5) - (6)].list); - n->behavior = (yyvsp[(6) - (6)].dbehavior); + n->objects = (yyvsp[-1].list); + n->behavior = (yyvsp[0].dbehavior); n->concurrent = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 22216 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 388: + case 405: /* DropStmt: DROP drop_type_any_name any_name_list opt_drop_behavior */ #line 20 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (4)].objtype); + n->removeType = (yyvsp[-2].objtype); n->missing_ok = false; - n->objects = (yyvsp[(3) - (4)].list); - n->behavior = (yyvsp[(4) - (4)].dbehavior); + n->objects = (yyvsp[-1].list); + n->behavior = (yyvsp[0].dbehavior); n->concurrent = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 22230 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 389: + case 406: /* DropStmt: DROP drop_type_name IF_P EXISTS name_list opt_drop_behavior */ #line 30 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (6)].objtype); + n->removeType = (yyvsp[-4].objtype); n->missing_ok = true; - n->objects = (yyvsp[(5) - (6)].list); - n->behavior = (yyvsp[(6) - (6)].dbehavior); + n->objects = (yyvsp[-1].list); + n->behavior = (yyvsp[0].dbehavior); n->concurrent = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 22244 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 390: + case 407: /* DropStmt: DROP drop_type_name name_list opt_drop_behavior */ #line 40 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (4)].objtype); + n->removeType = (yyvsp[-2].objtype); n->missing_ok = false; - n->objects = (yyvsp[(3) - (4)].list); - n->behavior = (yyvsp[(4) - (4)].dbehavior); + n->objects = (yyvsp[-1].list); + n->behavior = (yyvsp[0].dbehavior); n->concurrent = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 22258 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 391: + case 408: /* DropStmt: DROP drop_type_name_on_any_name name ON any_name opt_drop_behavior */ #line 50 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (6)].objtype); - n->objects = list_make1(lappend((yyvsp[(5) - (6)].list), makeString((yyvsp[(3) - (6)].str)))); - n->behavior = (yyvsp[(6) - (6)].dbehavior); + n->removeType = (yyvsp[-4].objtype); + n->objects = list_make1(lappend((yyvsp[-1].list), makeString((yyvsp[-3].str)))); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = false; n->concurrent = false; (yyval.node) = (PGNode *) n; - ;} + } +#line 22272 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 392: + case 409: /* DropStmt: DROP drop_type_name_on_any_name IF_P EXISTS name ON any_name opt_drop_behavior */ #line 60 "third_party/libpg_query/grammar/statements/drop.y" - { + { PGDropStmt *n = makeNode(PGDropStmt); - n->removeType = (yyvsp[(2) - (8)].objtype); - n->objects = list_make1(lappend((yyvsp[(7) - (8)].list), makeString((yyvsp[(5) - (8)].str)))); - n->behavior = (yyvsp[(8) - (8)].dbehavior); + n->removeType = (yyvsp[-6].objtype); + n->objects = list_make1(lappend((yyvsp[-1].list), makeString((yyvsp[-3].str)))); + n->behavior = (yyvsp[0].dbehavior); n->missing_ok = true; n->concurrent = false; (yyval.node) = (PGNode *) n; - ;} + } +#line 22286 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 393: + case 410: /* drop_type_any_name: TABLE */ #line 73 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TABLE; ;} + { (yyval.objtype) = PG_OBJECT_TABLE; } +#line 22292 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 394: + case 411: /* drop_type_any_name: SEQUENCE */ #line 74 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_SEQUENCE; ;} + { (yyval.objtype) = PG_OBJECT_SEQUENCE; } +#line 22298 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 395: + case 412: /* drop_type_any_name: FUNCTION */ #line 75 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_FUNCTION; ;} + { (yyval.objtype) = PG_OBJECT_FUNCTION; } +#line 22304 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 396: + case 413: /* drop_type_any_name: MACRO */ #line 76 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_FUNCTION; ;} + { (yyval.objtype) = PG_OBJECT_FUNCTION; } +#line 22310 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 397: + case 414: /* drop_type_any_name: MACRO TABLE */ #line 77 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TABLE_MACRO; ;} + { (yyval.objtype) = PG_OBJECT_TABLE_MACRO; } +#line 22316 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 398: + case 415: /* drop_type_any_name: VIEW */ #line 78 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_VIEW; ;} + { (yyval.objtype) = PG_OBJECT_VIEW; } +#line 22322 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 399: + case 416: /* drop_type_any_name: MATERIALIZED VIEW */ #line 79 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_MATVIEW; ;} + { (yyval.objtype) = PG_OBJECT_MATVIEW; } +#line 22328 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 400: + case 417: /* drop_type_any_name: INDEX */ #line 80 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_INDEX; ;} + { (yyval.objtype) = PG_OBJECT_INDEX; } +#line 22334 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 401: + case 418: /* drop_type_any_name: FOREIGN TABLE */ #line 81 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_FOREIGN_TABLE; ;} + { (yyval.objtype) = PG_OBJECT_FOREIGN_TABLE; } +#line 22340 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 402: + case 419: /* drop_type_any_name: COLLATION */ #line 82 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_COLLATION; ;} + { (yyval.objtype) = PG_OBJECT_COLLATION; } +#line 22346 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 403: + case 420: /* drop_type_any_name: CONVERSION_P */ #line 83 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_CONVERSION; ;} + { (yyval.objtype) = PG_OBJECT_CONVERSION; } +#line 22352 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 404: + case 421: /* drop_type_any_name: SCHEMA */ #line 84 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_SCHEMA; ;} + { (yyval.objtype) = PG_OBJECT_SCHEMA; } +#line 22358 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 405: + case 422: /* drop_type_any_name: STATISTICS */ #line 85 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_STATISTIC_EXT; ;} + { (yyval.objtype) = PG_OBJECT_STATISTIC_EXT; } +#line 22364 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 406: + case 423: /* drop_type_any_name: TEXT_P SEARCH PARSER */ #line 86 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TSPARSER; ;} + { (yyval.objtype) = PG_OBJECT_TSPARSER; } +#line 22370 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 407: + case 424: /* drop_type_any_name: TEXT_P SEARCH DICTIONARY */ #line 87 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TSDICTIONARY; ;} + { (yyval.objtype) = PG_OBJECT_TSDICTIONARY; } +#line 22376 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 408: + case 425: /* drop_type_any_name: TEXT_P SEARCH TEMPLATE */ #line 88 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TSTEMPLATE; ;} + { (yyval.objtype) = PG_OBJECT_TSTEMPLATE; } +#line 22382 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 409: + case 426: /* drop_type_any_name: TEXT_P SEARCH CONFIGURATION */ #line 89 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TSCONFIGURATION; ;} + { (yyval.objtype) = PG_OBJECT_TSCONFIGURATION; } +#line 22388 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 410: + case 427: /* drop_type_any_name: TYPE_P */ #line 90 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TYPE; ;} + { (yyval.objtype) = PG_OBJECT_TYPE; } +#line 22394 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 411: + case 428: /* drop_type_name: ACCESS METHOD */ #line 95 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_ACCESS_METHOD; ;} + { (yyval.objtype) = PG_OBJECT_ACCESS_METHOD; } +#line 22400 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 412: + case 429: /* drop_type_name: EVENT TRIGGER */ #line 96 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_EVENT_TRIGGER; ;} + { (yyval.objtype) = PG_OBJECT_EVENT_TRIGGER; } +#line 22406 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 413: + case 430: /* drop_type_name: EXTENSION */ #line 97 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_EXTENSION; ;} + { (yyval.objtype) = PG_OBJECT_EXTENSION; } +#line 22412 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 414: + case 431: /* drop_type_name: FOREIGN DATA_P WRAPPER */ #line 98 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_FDW; ;} + { (yyval.objtype) = PG_OBJECT_FDW; } +#line 22418 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 415: + case 432: /* drop_type_name: PUBLICATION */ #line 99 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_PUBLICATION; ;} + { (yyval.objtype) = PG_OBJECT_PUBLICATION; } +#line 22424 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 416: + case 433: /* drop_type_name: SERVER */ #line 100 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_FOREIGN_SERVER; ;} + { (yyval.objtype) = PG_OBJECT_FOREIGN_SERVER; } +#line 22430 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 417: + case 434: /* any_name_list: any_name */ #line 105 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} + { (yyval.list) = list_make1((yyvsp[0].list)); } +#line 22436 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 418: + case 435: /* any_name_list: any_name_list ',' any_name */ #line 106 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].list)); } +#line 22442 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 419: + case 436: /* opt_drop_behavior: CASCADE */ #line 111 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.dbehavior) = PG_DROP_CASCADE; ;} + { (yyval.dbehavior) = PG_DROP_CASCADE; } +#line 22448 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 420: + case 437: /* opt_drop_behavior: RESTRICT */ #line 112 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.dbehavior) = PG_DROP_RESTRICT; ;} + { (yyval.dbehavior) = PG_DROP_RESTRICT; } +#line 22454 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 421: + case 438: /* opt_drop_behavior: %empty */ #line 113 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.dbehavior) = PG_DROP_RESTRICT; /* default */ ;} + { (yyval.dbehavior) = PG_DROP_RESTRICT; /* default */ } +#line 22460 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 422: + case 439: /* drop_type_name_on_any_name: POLICY */ #line 118 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_POLICY; ;} + { (yyval.objtype) = PG_OBJECT_POLICY; } +#line 22466 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 423: + case 440: /* drop_type_name_on_any_name: RULE */ #line 119 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_RULE; ;} + { (yyval.objtype) = PG_OBJECT_RULE; } +#line 22472 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 424: + case 441: /* drop_type_name_on_any_name: TRIGGER */ #line 120 "third_party/libpg_query/grammar/statements/drop.y" - { (yyval.objtype) = PG_OBJECT_TRIGGER; ;} + { (yyval.objtype) = PG_OBJECT_TRIGGER; } +#line 22478 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 425: + case 442: /* CreateFunctionStmt: CREATE_P OptTemp macro_alias qualified_name param_list AS TABLE SelectStmt */ #line 9 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(4) - (8)].range)->relpersistence = (yyvsp[(2) - (8)].ival); - n->name = (yyvsp[(4) - (8)].range); - n->params = (yyvsp[(5) - (8)].list); + (yyvsp[-4].range)->relpersistence = (yyvsp[-6].ival); + n->name = (yyvsp[-4].range); + n->params = (yyvsp[-3].list); n->function = NULL; - n->query = (yyvsp[(8) - (8)].node); + n->query = (yyvsp[0].node); n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22493 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 426: + case 443: /* CreateFunctionStmt: CREATE_P OptTemp macro_alias IF_P NOT EXISTS qualified_name param_list AS TABLE SelectStmt */ #line 21 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(7) - (11)].range)->relpersistence = (yyvsp[(2) - (11)].ival); - n->name = (yyvsp[(7) - (11)].range); - n->params = (yyvsp[(8) - (11)].list); + (yyvsp[-4].range)->relpersistence = (yyvsp[-9].ival); + n->name = (yyvsp[-4].range); + n->params = (yyvsp[-3].list); n->function = NULL; - n->query = (yyvsp[(11) - (11)].node); + n->query = (yyvsp[0].node); n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22509 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 427: + case 444: /* CreateFunctionStmt: CREATE_P OR REPLACE OptTemp macro_alias qualified_name param_list AS TABLE SelectStmt */ #line 34 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(6) - (10)].range)->relpersistence = (yyvsp[(4) - (10)].ival); - n->name = (yyvsp[(6) - (10)].range); - n->params = (yyvsp[(7) - (10)].list); + (yyvsp[-4].range)->relpersistence = (yyvsp[-6].ival); + n->name = (yyvsp[-4].range); + n->params = (yyvsp[-3].list); n->function = NULL; - n->query = (yyvsp[(10) - (10)].node); + n->query = (yyvsp[0].node); n->onconflict = PG_REPLACE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22525 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 428: + case 445: /* CreateFunctionStmt: CREATE_P OptTemp macro_alias qualified_name param_list AS a_expr */ #line 47 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(4) - (7)].range)->relpersistence = (yyvsp[(2) - (7)].ival); - n->name = (yyvsp[(4) - (7)].range); - n->params = (yyvsp[(5) - (7)].list); - n->function = (yyvsp[(7) - (7)].node); + (yyvsp[-3].range)->relpersistence = (yyvsp[-5].ival); + n->name = (yyvsp[-3].range); + n->params = (yyvsp[-2].list); + n->function = (yyvsp[0].node); n->query = NULL; n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22540 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 429: + case 446: /* CreateFunctionStmt: CREATE_P OptTemp macro_alias IF_P NOT EXISTS qualified_name param_list AS a_expr */ #line 59 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(7) - (10)].range)->relpersistence = (yyvsp[(2) - (10)].ival); - n->name = (yyvsp[(7) - (10)].range); - n->params = (yyvsp[(8) - (10)].list); - n->function = (yyvsp[(10) - (10)].node); + (yyvsp[-3].range)->relpersistence = (yyvsp[-8].ival); + n->name = (yyvsp[-3].range); + n->params = (yyvsp[-2].list); + n->function = (yyvsp[0].node); n->query = NULL; n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22555 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 430: + case 447: /* CreateFunctionStmt: CREATE_P OR REPLACE OptTemp macro_alias qualified_name param_list AS a_expr */ #line 71 "third_party/libpg_query/grammar/statements/create_function.y" - { + { PGCreateFunctionStmt *n = makeNode(PGCreateFunctionStmt); - (yyvsp[(6) - (9)].range)->relpersistence = (yyvsp[(4) - (9)].ival); - n->name = (yyvsp[(6) - (9)].range); - n->params = (yyvsp[(7) - (9)].list); - n->function = (yyvsp[(9) - (9)].node); + (yyvsp[-3].range)->relpersistence = (yyvsp[-5].ival); + n->name = (yyvsp[-3].range); + n->params = (yyvsp[-2].list); + n->function = (yyvsp[0].node); n->query = NULL; n->onconflict = PG_REPLACE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 22570 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 433: + case 450: /* param_list: '(' ')' */ #line 92 "third_party/libpg_query/grammar/statements/create_function.y" - { + { (yyval.list) = NIL; - ;} + } +#line 22578 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 434: + case 451: /* param_list: '(' func_arg_list ')' */ #line 96 "third_party/libpg_query/grammar/statements/create_function.y" - { - (yyval.list) = (yyvsp[(2) - (3)].list); - ;} + { + (yyval.list) = (yyvsp[-1].list); + } +#line 22586 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 435: + case 452: /* UpdateStmt: opt_with_clause UPDATE relation_expr_opt_alias SET set_clause_list_opt_comma from_clause where_or_current_clause returning_clause */ #line 12 "third_party/libpg_query/grammar/statements/update.y" - { + { PGUpdateStmt *n = makeNode(PGUpdateStmt); - n->relation = (yyvsp[(3) - (8)].range); - n->targetList = (yyvsp[(5) - (8)].list); - n->fromClause = (yyvsp[(6) - (8)].list); - n->whereClause = (yyvsp[(7) - (8)].node); - n->returningList = (yyvsp[(8) - (8)].list); - n->withClause = (yyvsp[(1) - (8)].with); + n->relation = (yyvsp[-5].range); + n->targetList = (yyvsp[-3].list); + n->fromClause = (yyvsp[-2].list); + n->whereClause = (yyvsp[-1].node); + n->returningList = (yyvsp[0].list); + n->withClause = (yyvsp[-7].with); (yyval.node) = (PGNode *)n; - ;} + } +#line 22601 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 436: + case 453: /* CopyStmt: COPY opt_binary qualified_name opt_column_list opt_oids copy_from opt_program copy_file_name copy_delimiter opt_with copy_options */ #line 3 "third_party/libpg_query/grammar/statements/copy.y" - { + { PGCopyStmt *n = makeNode(PGCopyStmt); - n->relation = (yyvsp[(3) - (11)].range); + n->relation = (yyvsp[-8].range); n->query = NULL; - n->attlist = (yyvsp[(4) - (11)].list); - n->is_from = (yyvsp[(6) - (11)].boolean); - n->is_program = (yyvsp[(7) - (11)].boolean); - n->filename = (yyvsp[(8) - (11)].str); + n->attlist = (yyvsp[-7].list); + n->is_from = (yyvsp[-5].boolean); + n->is_program = (yyvsp[-4].boolean); + n->filename = (yyvsp[-3].str); if (n->is_program && n->filename == NULL) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("STDIN/STDOUT not allowed with PROGRAM"), - parser_errposition((yylsp[(8) - (11)])))); + parser_errposition((yylsp[-3])))); n->options = NIL; /* Concatenate user-supplied flags */ - if ((yyvsp[(2) - (11)].defelt)) - n->options = lappend(n->options, (yyvsp[(2) - (11)].defelt)); - if ((yyvsp[(5) - (11)].defelt)) - n->options = lappend(n->options, (yyvsp[(5) - (11)].defelt)); - if ((yyvsp[(9) - (11)].defelt)) - n->options = lappend(n->options, (yyvsp[(9) - (11)].defelt)); - if ((yyvsp[(11) - (11)].list)) - n->options = list_concat(n->options, (yyvsp[(11) - (11)].list)); + if ((yyvsp[-9].defelt)) + n->options = lappend(n->options, (yyvsp[-9].defelt)); + if ((yyvsp[-6].defelt)) + n->options = lappend(n->options, (yyvsp[-6].defelt)); + if ((yyvsp[-2].defelt)) + n->options = lappend(n->options, (yyvsp[-2].defelt)); + if ((yyvsp[0].list)) + n->options = list_concat(n->options, (yyvsp[0].list)); (yyval.node) = (PGNode *)n; - ;} + } +#line 22633 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 437: + case 454: /* CopyStmt: COPY '(' SelectStmt ')' TO opt_program copy_file_name opt_with copy_options */ #line 31 "third_party/libpg_query/grammar/statements/copy.y" - { + { PGCopyStmt *n = makeNode(PGCopyStmt); n->relation = NULL; - n->query = (yyvsp[(3) - (9)].node); + n->query = (yyvsp[-6].node); n->attlist = NIL; n->is_from = false; - n->is_program = (yyvsp[(6) - (9)].boolean); - n->filename = (yyvsp[(7) - (9)].str); - n->options = (yyvsp[(9) - (9)].list); + n->is_program = (yyvsp[-3].boolean); + n->filename = (yyvsp[-2].str); + n->options = (yyvsp[0].list); if (n->is_program && n->filename == NULL) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("STDIN/STDOUT not allowed with PROGRAM"), - parser_errposition((yylsp[(5) - (9)])))); + parser_errposition((yylsp[-4])))); (yyval.node) = (PGNode *)n; - ;} + } +#line 22656 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 438: + case 455: /* CopyStmt: COPY FROM DATABASE ColId TO ColId copy_database_flag */ #line 50 "third_party/libpg_query/grammar/statements/copy.y" - { + { PGCopyDatabaseStmt *n = makeNode(PGCopyDatabaseStmt); - n->from_database = (yyvsp[(4) - (7)].str); - n->to_database = (yyvsp[(6) - (7)].str); - n->copy_database_flag = (yyvsp[(7) - (7)].conststr); + n->from_database = (yyvsp[-3].str); + n->to_database = (yyvsp[-1].str); + n->copy_database_flag = (yyvsp[0].conststr); (yyval.node) = (PGNode *)n; - ;} + } +#line 22668 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 439: + case 456: /* copy_database_flag: %empty */ #line 61 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.conststr) = NULL; ;} + { (yyval.conststr) = NULL; } +#line 22674 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 440: + case 457: /* copy_database_flag: '(' SCHEMA ')' */ #line 62 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.conststr) = "schema"; ;} + { (yyval.conststr) = "schema"; } +#line 22680 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 441: + case 458: /* copy_database_flag: '(' DATA_P ')' */ #line 63 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.conststr) = "data"; ;} + { (yyval.conststr) = "data"; } +#line 22686 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 442: + case 459: /* copy_from: FROM */ #line 67 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 22692 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 443: + case 460: /* copy_from: TO */ #line 68 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 22698 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 444: + case 461: /* copy_delimiter: opt_using DELIMITERS Sconst */ #line 74 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("delimiter", (PGNode *)makeString((yyvsp[(3) - (3)].str)), (yylsp[(2) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("delimiter", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-1])); + } +#line 22706 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 445: + case 462: /* copy_delimiter: %empty */ #line 77 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.defelt) = NULL; ;} + { (yyval.defelt) = NULL; } +#line 22712 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 446: + case 463: /* copy_generic_opt_arg_list: copy_generic_opt_arg_list_item */ #line 83 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 22720 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 447: + case 464: /* copy_generic_opt_arg_list: copy_generic_opt_arg_list ',' copy_generic_opt_arg_list_item */ #line 87 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 22728 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 448: + case 465: /* opt_using: USING */ #line 94 "third_party/libpg_query/grammar/statements/copy.y" - {;} + {} +#line 22734 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 449: + case 466: /* opt_using: %empty */ #line 95 "third_party/libpg_query/grammar/statements/copy.y" - {;} + {} +#line 22740 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 450: + case 467: /* opt_as: AS */ #line 99 "third_party/libpg_query/grammar/statements/copy.y" - {;} + {} +#line 22746 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 451: + case 468: /* opt_as: %empty */ #line 100 "third_party/libpg_query/grammar/statements/copy.y" - {;} + {} +#line 22752 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 452: + case 469: /* opt_program: PROGRAM */ #line 105 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 22758 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 453: + case 470: /* opt_program: %empty */ #line 106 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 22764 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 454: + case 471: /* copy_options: copy_opt_list */ #line 110 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 22770 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 455: + case 472: /* copy_options: '(' copy_generic_opt_list ')' */ #line 111 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 22776 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 456: + case 473: /* copy_generic_opt_arg: opt_boolean_or_string */ #line 116 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); ;} + { (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); } +#line 22782 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 457: + case 474: /* copy_generic_opt_arg: NumericOnly */ #line 117 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) (yyvsp[(1) - (1)].value); ;} + { (yyval.node) = (PGNode *) (yyvsp[0].value); } +#line 22788 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 458: + case 475: /* copy_generic_opt_arg: '*' */ #line 118 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) makeNode(PGAStar); ;} + { (yyval.node) = (PGNode *) makeNode(PGAStar); } +#line 22794 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 459: + case 476: /* copy_generic_opt_arg: '(' copy_generic_opt_arg_list ')' */ #line 119 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) (yyvsp[(2) - (3)].list); ;} + { (yyval.node) = (PGNode *) (yyvsp[-1].list); } +#line 22800 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 460: + case 477: /* copy_generic_opt_arg: struct_expr */ #line 120 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (PGNode *) (yyvsp[0].node); } +#line 22806 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 461: + case 478: /* copy_generic_opt_arg: %empty */ #line 121 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 22812 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 462: + case 479: /* copy_generic_opt_elem: ColLabel copy_generic_opt_arg */ #line 127 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[-1].str), (yyvsp[0].node), (yylsp[-1])); + } +#line 22820 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 463: + case 480: /* opt_oids: WITH OIDS */ #line 135 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("oids", (PGNode *)makeInteger(true), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("oids", (PGNode *)makeInteger(true), (yylsp[-1])); + } +#line 22828 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 464: + case 481: /* opt_oids: %empty */ #line 138 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.defelt) = NULL; ;} + { (yyval.defelt) = NULL; } +#line 22834 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 465: + case 482: /* copy_opt_list: copy_opt_list copy_opt_item */ #line 143 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].defelt)); ;} + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].defelt)); } +#line 22840 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 466: + case 483: /* copy_opt_list: %empty */ #line 144 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 22846 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 467: + case 484: /* opt_binary: BINARY */ #line 150 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("binary"), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("binary"), (yylsp[0])); + } +#line 22854 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 468: + case 485: /* opt_binary: %empty */ #line 153 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.defelt) = NULL; ;} + { (yyval.defelt) = NULL; } +#line 22860 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 469: + case 486: /* copy_opt_item: BINARY */ #line 159 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("binary"), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("binary"), (yylsp[0])); + } +#line 22868 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 470: + case 487: /* copy_opt_item: OIDS */ #line 163 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("oids", (PGNode *)makeInteger(true), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("oids", (PGNode *)makeInteger(true), (yylsp[0])); + } +#line 22876 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 471: + case 488: /* copy_opt_item: FREEZE */ #line 167 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("freeze", (PGNode *)makeInteger(true), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("freeze", (PGNode *)makeInteger(true), (yylsp[0])); + } +#line 22884 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 472: + case 489: /* copy_opt_item: DELIMITER opt_as Sconst */ #line 171 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("delimiter", (PGNode *)makeString((yyvsp[(3) - (3)].str)), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("delimiter", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-2])); + } +#line 22892 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 473: + case 490: /* copy_opt_item: NULL_P opt_as Sconst */ #line 175 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("null", (PGNode *)makeString((yyvsp[(3) - (3)].str)), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("null", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-2])); + } +#line 22900 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 474: + case 491: /* copy_opt_item: CSV */ #line 179 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("csv"), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("format", (PGNode *)makeString("csv"), (yylsp[0])); + } +#line 22908 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 475: + case 492: /* copy_opt_item: HEADER_P */ #line 183 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("header", (PGNode *)makeInteger(true), (yylsp[(1) - (1)])); - ;} + { + (yyval.defelt) = makeDefElem("header", (PGNode *)makeInteger(true), (yylsp[0])); + } +#line 22916 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 476: + case 493: /* copy_opt_item: QUOTE opt_as Sconst */ #line 187 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("quote", (PGNode *)makeString((yyvsp[(3) - (3)].str)), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("quote", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-2])); + } +#line 22924 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 477: + case 494: /* copy_opt_item: ESCAPE opt_as Sconst */ #line 191 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("escape", (PGNode *)makeString((yyvsp[(3) - (3)].str)), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("escape", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-2])); + } +#line 22932 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 478: + case 495: /* copy_opt_item: FORCE QUOTE columnList */ #line 195 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("force_quote", (PGNode *)(yyvsp[(3) - (3)].list), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("force_quote", (PGNode *)(yyvsp[0].list), (yylsp[-2])); + } +#line 22940 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 479: + case 496: /* copy_opt_item: FORCE QUOTE '*' */ #line 199 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("force_quote", (PGNode *)makeNode(PGAStar), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("force_quote", (PGNode *)makeNode(PGAStar), (yylsp[-2])); + } +#line 22948 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 480: + case 497: /* copy_opt_item: PARTITION BY columnList */ #line 203 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("partition_by", (PGNode *)(yyvsp[(3) - (3)].list), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("partition_by", (PGNode *)(yyvsp[0].list), (yylsp[-2])); + } +#line 22956 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 481: + case 498: /* copy_opt_item: PARTITION BY '*' */ #line 207 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("partition_by", (PGNode *)makeNode(PGAStar), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("partition_by", (PGNode *)makeNode(PGAStar), (yylsp[-2])); + } +#line 22964 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 482: + case 499: /* copy_opt_item: FORCE NOT NULL_P columnList */ #line 211 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("force_not_null", (PGNode *)(yyvsp[(4) - (4)].list), (yylsp[(1) - (4)])); - ;} + { + (yyval.defelt) = makeDefElem("force_not_null", (PGNode *)(yyvsp[0].list), (yylsp[-3])); + } +#line 22972 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 483: + case 500: /* copy_opt_item: FORCE NULL_P columnList */ #line 215 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("force_null", (PGNode *)(yyvsp[(3) - (3)].list), (yylsp[(1) - (3)])); - ;} + { + (yyval.defelt) = makeDefElem("force_null", (PGNode *)(yyvsp[0].list), (yylsp[-2])); + } +#line 22980 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 484: + case 501: /* copy_opt_item: ENCODING Sconst */ #line 219 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.defelt) = makeDefElem("encoding", (PGNode *)makeString((yyvsp[(2) - (2)].str)), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem("encoding", (PGNode *)makeString((yyvsp[0].str)), (yylsp[-1])); + } +#line 22988 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 485: + case 502: /* copy_generic_opt_arg_list_item: opt_boolean_or_string */ #line 226 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); ;} + { (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); } +#line 22994 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 486: + case 503: /* copy_file_name: Sconst */ #line 231 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 23000 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 487: + case 504: /* copy_file_name: STDIN */ #line 232 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.str) = NULL; ;} + { (yyval.str) = NULL; } +#line 23006 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 488: + case 505: /* copy_file_name: STDOUT */ #line 233 "third_party/libpg_query/grammar/statements/copy.y" - { (yyval.str) = NULL; ;} + { (yyval.str) = NULL; } +#line 23012 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 489: -#line 239 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); - ;} + case 506: /* copy_file_name: IDENT '.' ColId */ +#line 234 "third_party/libpg_query/grammar/statements/copy.y" + { (yyval.str) = psprintf("%s.%s", (yyvsp[-2].str), (yyvsp[0].str)); } +#line 23018 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 490: -#line 243 "third_party/libpg_query/grammar/statements/copy.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); - ;} + case 507: /* copy_file_name: IDENT */ +#line 235 "third_party/libpg_query/grammar/statements/copy.y" + { (yyval.str) = (yyvsp[0].str); } +#line 23024 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 493: + case 508: /* copy_generic_opt_list: copy_generic_opt_elem */ +#line 242 "third_party/libpg_query/grammar/statements/copy.y" + { + (yyval.list) = list_make1((yyvsp[0].defelt)); + } +#line 23032 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 509: /* copy_generic_opt_list: copy_generic_opt_list ',' copy_generic_opt_elem */ +#line 246 "third_party/libpg_query/grammar/statements/copy.y" + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].defelt)); + } +#line 23040 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 512: /* select_with_parens: '(' select_no_parens ')' */ #line 52 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (3)].node); ;} + { (yyval.node) = (yyvsp[-1].node); } +#line 23046 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 494: + case 513: /* select_with_parens: '(' select_with_parens ')' */ #line 53 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (3)].node); ;} + { (yyval.node) = (yyvsp[-1].node); } +#line 23052 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 495: -#line 68 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 514: /* select_with_parens: '(' VariableShowStmt ')' */ +#line 55 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (yyvsp[-1].node); + } +#line 23060 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 496: -#line 70 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(1) - (2)].node), (yyvsp[(2) - (2)].list), NIL, + case 515: /* select_no_parens: simple_select */ +#line 72 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23066 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 516: /* select_no_parens: select_clause sort_clause */ +#line 74 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-1].node), (yyvsp[0].list), NIL, NULL, NULL, NULL, yyscanner); - (yyval.node) = (yyvsp[(1) - (2)].node); - ;} + (yyval.node) = (yyvsp[-1].node); + } +#line 23077 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 497: -#line 77 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(1) - (4)].node), (yyvsp[(2) - (4)].list), (yyvsp[(3) - (4)].list), - (PGNode*) list_nth((yyvsp[(4) - (4)].list), 0), (PGNode*) list_nth((yyvsp[(4) - (4)].list), 1), + case 517: /* select_no_parens: select_clause opt_sort_clause for_locking_clause opt_select_limit */ +#line 81 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-3].node), (yyvsp[-2].list), (yyvsp[-1].list), + (PGNode*) list_nth((yyvsp[0].list), 0), (PGNode*) list_nth((yyvsp[0].list), 1), NULL, yyscanner); - (yyval.node) = (yyvsp[(1) - (4)].node); - ;} + (yyval.node) = (yyvsp[-3].node); + } +#line 23089 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 498: -#line 85 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(1) - (4)].node), (yyvsp[(2) - (4)].list), (yyvsp[(4) - (4)].list), - (PGNode*) list_nth((yyvsp[(3) - (4)].list), 0), (PGNode*) list_nth((yyvsp[(3) - (4)].list), 1), + case 518: /* select_no_parens: select_clause opt_sort_clause select_limit opt_for_locking_clause */ +#line 89 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-3].node), (yyvsp[-2].list), (yyvsp[0].list), + (PGNode*) list_nth((yyvsp[-1].list), 0), (PGNode*) list_nth((yyvsp[-1].list), 1), NULL, yyscanner); - (yyval.node) = (yyvsp[(1) - (4)].node); - ;} + (yyval.node) = (yyvsp[-3].node); + } +#line 23101 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 499: -#line 93 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(2) - (2)].node), NULL, NIL, + case 519: /* select_no_parens: with_clause select_clause */ +#line 97 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[0].node), NULL, NIL, NULL, NULL, - (yyvsp[(1) - (2)].with), + (yyvsp[-1].with), yyscanner); - (yyval.node) = (yyvsp[(2) - (2)].node); - ;} + (yyval.node) = (yyvsp[0].node); + } +#line 23113 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 500: -#line 101 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(2) - (3)].node), (yyvsp[(3) - (3)].list), NIL, + case 520: /* select_no_parens: with_clause select_clause sort_clause */ +#line 105 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-1].node), (yyvsp[0].list), NIL, NULL, NULL, - (yyvsp[(1) - (3)].with), + (yyvsp[-2].with), yyscanner); - (yyval.node) = (yyvsp[(2) - (3)].node); - ;} + (yyval.node) = (yyvsp[-1].node); + } +#line 23125 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 501: -#line 109 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(2) - (5)].node), (yyvsp[(3) - (5)].list), (yyvsp[(4) - (5)].list), - (PGNode*) list_nth((yyvsp[(5) - (5)].list), 0), (PGNode*) list_nth((yyvsp[(5) - (5)].list), 1), - (yyvsp[(1) - (5)].with), + case 521: /* select_no_parens: with_clause select_clause opt_sort_clause for_locking_clause opt_select_limit */ +#line 113 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-3].node), (yyvsp[-2].list), (yyvsp[-1].list), + (PGNode*) list_nth((yyvsp[0].list), 0), (PGNode*) list_nth((yyvsp[0].list), 1), + (yyvsp[-4].with), yyscanner); - (yyval.node) = (yyvsp[(2) - (5)].node); - ;} + (yyval.node) = (yyvsp[-3].node); + } +#line 23137 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 502: -#line 117 "third_party/libpg_query/grammar/statements/select.y" - { - insertSelectOptions((PGSelectStmt *) (yyvsp[(2) - (5)].node), (yyvsp[(3) - (5)].list), (yyvsp[(5) - (5)].list), - (PGNode*) list_nth((yyvsp[(4) - (5)].list), 0), (PGNode*) list_nth((yyvsp[(4) - (5)].list), 1), - (yyvsp[(1) - (5)].with), + case 522: /* select_no_parens: with_clause select_clause opt_sort_clause select_limit opt_for_locking_clause */ +#line 121 "third_party/libpg_query/grammar/statements/select.y" + { + insertSelectOptions((PGSelectStmt *) (yyvsp[-3].node), (yyvsp[-2].list), (yyvsp[0].list), + (PGNode*) list_nth((yyvsp[-1].list), 0), (PGNode*) list_nth((yyvsp[-1].list), 1), + (yyvsp[-4].with), yyscanner); - (yyval.node) = (yyvsp[(2) - (5)].node); - ;} + (yyval.node) = (yyvsp[-3].node); + } +#line 23149 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 503: -#line 127 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 523: /* select_clause: simple_select */ +#line 131 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23155 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 504: -#line 128 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 524: /* select_clause: select_with_parens */ +#line 132 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23161 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 505: -#line 156 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(3) - (3)].list); - ;} + case 525: /* opt_select: SELECT opt_all_clause opt_target_list_opt_comma */ +#line 160 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = (yyvsp[0].list); + } +#line 23169 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 506: -#line 160 "third_party/libpg_query/grammar/statements/select.y" - { + case 526: /* opt_select: %empty */ +#line 164 "third_party/libpg_query/grammar/statements/select.y" + { PGAStar *star = makeNode(PGAStar); (yyval.list) = list_make1(star); - ;} + } +#line 23178 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 507: -#line 171 "third_party/libpg_query/grammar/statements/select.y" - { + case 527: /* simple_select: SELECT opt_all_clause opt_target_list_opt_comma into_clause from_clause where_clause group_clause having_clause window_clause qualify_clause sample_clause */ +#line 175 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *n = makeNode(PGSelectStmt); - n->targetList = (yyvsp[(3) - (11)].list); - n->intoClause = (yyvsp[(4) - (11)].into); - n->fromClause = (yyvsp[(5) - (11)].list); - n->whereClause = (yyvsp[(6) - (11)].node); - n->groupClause = (yyvsp[(7) - (11)].list); - n->havingClause = (yyvsp[(8) - (11)].node); - n->windowClause = (yyvsp[(9) - (11)].list); - n->qualifyClause = (yyvsp[(10) - (11)].node); - n->sampleOptions = (yyvsp[(11) - (11)].node); + n->targetList = (yyvsp[-8].list); + n->intoClause = (yyvsp[-7].into); + n->fromClause = (yyvsp[-6].list); + n->whereClause = (yyvsp[-5].node); + n->groupClause = (yyvsp[-4].list); + n->havingClause = (yyvsp[-3].node); + n->windowClause = (yyvsp[-2].list); + n->qualifyClause = (yyvsp[-1].node); + n->sampleOptions = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 23196 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 508: -#line 187 "third_party/libpg_query/grammar/statements/select.y" - { + case 528: /* simple_select: SELECT distinct_clause target_list_opt_comma into_clause from_clause where_clause group_clause having_clause window_clause qualify_clause sample_clause */ +#line 191 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *n = makeNode(PGSelectStmt); - n->distinctClause = (yyvsp[(2) - (11)].list); - n->targetList = (yyvsp[(3) - (11)].list); - n->intoClause = (yyvsp[(4) - (11)].into); - n->fromClause = (yyvsp[(5) - (11)].list); - n->whereClause = (yyvsp[(6) - (11)].node); - n->groupClause = (yyvsp[(7) - (11)].list); - n->havingClause = (yyvsp[(8) - (11)].node); - n->windowClause = (yyvsp[(9) - (11)].list); - n->qualifyClause = (yyvsp[(10) - (11)].node); - n->sampleOptions = (yyvsp[(11) - (11)].node); + n->distinctClause = (yyvsp[-9].list); + n->targetList = (yyvsp[-8].list); + n->intoClause = (yyvsp[-7].into); + n->fromClause = (yyvsp[-6].list); + n->whereClause = (yyvsp[-5].node); + n->groupClause = (yyvsp[-4].list); + n->havingClause = (yyvsp[-3].node); + n->windowClause = (yyvsp[-2].list); + n->qualifyClause = (yyvsp[-1].node); + n->sampleOptions = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 23215 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 509: -#line 204 "third_party/libpg_query/grammar/statements/select.y" - { + case 529: /* simple_select: FROM from_list opt_select into_clause where_clause group_clause having_clause window_clause qualify_clause sample_clause */ +#line 208 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *n = makeNode(PGSelectStmt); - n->targetList = (yyvsp[(3) - (10)].list); - n->fromClause = (yyvsp[(2) - (10)].list); - n->intoClause = (yyvsp[(4) - (10)].into); - n->whereClause = (yyvsp[(5) - (10)].node); - n->groupClause = (yyvsp[(6) - (10)].list); - n->havingClause = (yyvsp[(7) - (10)].node); - n->windowClause = (yyvsp[(8) - (10)].list); - n->qualifyClause = (yyvsp[(9) - (10)].node); - n->sampleOptions = (yyvsp[(10) - (10)].node); + n->targetList = (yyvsp[-7].list); + n->fromClause = (yyvsp[-8].list); + n->intoClause = (yyvsp[-6].into); + n->whereClause = (yyvsp[-5].node); + n->groupClause = (yyvsp[-4].list); + n->havingClause = (yyvsp[-3].node); + n->windowClause = (yyvsp[-2].list); + n->qualifyClause = (yyvsp[-1].node); + n->sampleOptions = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 23233 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 510: -#line 221 "third_party/libpg_query/grammar/statements/select.y" - { + case 530: /* simple_select: FROM from_list SELECT distinct_clause target_list_opt_comma into_clause where_clause group_clause having_clause window_clause qualify_clause sample_clause */ +#line 225 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *n = makeNode(PGSelectStmt); - n->targetList = (yyvsp[(5) - (12)].list); - n->distinctClause = (yyvsp[(4) - (12)].list); - n->fromClause = (yyvsp[(2) - (12)].list); - n->intoClause = (yyvsp[(6) - (12)].into); - n->whereClause = (yyvsp[(7) - (12)].node); - n->groupClause = (yyvsp[(8) - (12)].list); - n->havingClause = (yyvsp[(9) - (12)].node); - n->windowClause = (yyvsp[(10) - (12)].list); - n->qualifyClause = (yyvsp[(11) - (12)].node); - n->sampleOptions = (yyvsp[(12) - (12)].node); + n->targetList = (yyvsp[-7].list); + n->distinctClause = (yyvsp[-8].list); + n->fromClause = (yyvsp[-10].list); + n->intoClause = (yyvsp[-6].into); + n->whereClause = (yyvsp[-5].node); + n->groupClause = (yyvsp[-4].list); + n->havingClause = (yyvsp[-3].node); + n->windowClause = (yyvsp[-2].list); + n->qualifyClause = (yyvsp[-1].node); + n->sampleOptions = (yyvsp[0].node); (yyval.node) = (PGNode *)n; - ;} + } +#line 23252 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 511: -#line 235 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 531: /* simple_select: values_clause_opt_comma */ +#line 239 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23258 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 512: -#line 237 "third_party/libpg_query/grammar/statements/select.y" - { + case 532: /* simple_select: TABLE relation_expr */ +#line 241 "third_party/libpg_query/grammar/statements/select.y" + { /* same as SELECT * FROM relation_expr */ PGColumnRef *cr = makeNode(PGColumnRef); PGResTarget *rt = makeNode(PGResTarget); @@ -23019,1734 +23274,1973 @@ YYLTYPE yylloc; rt->location = -1; n->targetList = list_make1(rt); - n->fromClause = list_make1((yyvsp[(2) - (2)].range)); + n->fromClause = list_make1((yyvsp[0].range)); (yyval.node) = (PGNode *)n; - ;} - break; - - case 513: -#line 256 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSetOp(PG_SETOP_UNION_BY_NAME, (yyvsp[(3) - (5)].boolean), (yyvsp[(1) - (5)].node), (yyvsp[(5) - (5)].node)); - ;} + } +#line 23281 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 514: + case 533: /* simple_select: select_clause UNION all_or_distinct by_name select_clause */ #line 260 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSetOp(PG_SETOP_UNION, (yyvsp[(3) - (4)].boolean), (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node)); - ;} + { + (yyval.node) = makeSetOp(PG_SETOP_UNION_BY_NAME, (yyvsp[-2].boolean), (yyvsp[-4].node), (yyvsp[0].node)); + } +#line 23289 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 515: + case 534: /* simple_select: select_clause UNION all_or_distinct select_clause */ #line 264 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSetOp(PG_SETOP_INTERSECT, (yyvsp[(3) - (4)].boolean), (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node)); - ;} + { + (yyval.node) = makeSetOp(PG_SETOP_UNION, (yyvsp[-1].boolean), (yyvsp[-3].node), (yyvsp[0].node)); + } +#line 23297 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 516: + case 535: /* simple_select: select_clause INTERSECT all_or_distinct select_clause */ #line 268 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSetOp(PG_SETOP_EXCEPT, (yyvsp[(3) - (4)].boolean), (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node)); - ;} + { + (yyval.node) = makeSetOp(PG_SETOP_INTERSECT, (yyvsp[-1].boolean), (yyvsp[-3].node), (yyvsp[0].node)); + } +#line 23305 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 517: + case 536: /* simple_select: select_clause EXCEPT all_or_distinct select_clause */ #line 272 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.node) = makeSetOp(PG_SETOP_EXCEPT, (yyvsp[-1].boolean), (yyvsp[-3].node), (yyvsp[0].node)); + } +#line 23313 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 537: /* simple_select: pivot_keyword table_ref USING target_list_opt_comma */ +#line 276 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (4)].node); - n->aggrs = (yyvsp[(4) - (4)].list); + n->source = (yyvsp[-2].node); + n->aggrs = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23326 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 518: -#line 281 "third_party/libpg_query/grammar/statements/select.y" - { + case 538: /* simple_select: pivot_keyword table_ref USING target_list_opt_comma GROUP_P BY name_list_opt_comma_opt_bracket */ +#line 285 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (7)].node); - n->aggrs = (yyvsp[(4) - (7)].list); - n->groups = (yyvsp[(7) - (7)].list); + n->source = (yyvsp[-5].node); + n->aggrs = (yyvsp[-3].list); + n->groups = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23340 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 519: -#line 291 "third_party/libpg_query/grammar/statements/select.y" - { + case 539: /* simple_select: pivot_keyword table_ref GROUP_P BY name_list_opt_comma_opt_bracket */ +#line 295 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (5)].node); - n->groups = (yyvsp[(5) - (5)].list); + n->source = (yyvsp[-3].node); + n->groups = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23353 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 520: -#line 300 "third_party/libpg_query/grammar/statements/select.y" - { + case 540: /* simple_select: pivot_keyword table_ref ON pivot_column_list */ +#line 304 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (4)].node); - n->columns = (yyvsp[(4) - (4)].list); + n->source = (yyvsp[-2].node); + n->columns = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23366 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 521: -#line 309 "third_party/libpg_query/grammar/statements/select.y" - { + case 541: /* simple_select: pivot_keyword table_ref ON pivot_column_list GROUP_P BY name_list_opt_comma_opt_bracket */ +#line 313 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (7)].node); - n->columns = (yyvsp[(4) - (7)].list); - n->groups = (yyvsp[(7) - (7)].list); + n->source = (yyvsp[-5].node); + n->columns = (yyvsp[-3].list); + n->groups = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23380 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 522: -#line 319 "third_party/libpg_query/grammar/statements/select.y" - { + case 542: /* simple_select: pivot_keyword table_ref ON pivot_column_list USING target_list_opt_comma */ +#line 323 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (6)].node); - n->columns = (yyvsp[(4) - (6)].list); - n->aggrs = (yyvsp[(6) - (6)].list); + n->source = (yyvsp[-4].node); + n->columns = (yyvsp[-2].list); + n->aggrs = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23394 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 523: -#line 329 "third_party/libpg_query/grammar/statements/select.y" - { + case 543: /* simple_select: pivot_keyword table_ref ON pivot_column_list USING target_list_opt_comma GROUP_P BY name_list_opt_comma_opt_bracket */ +#line 333 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (9)].node); - n->columns = (yyvsp[(4) - (9)].list); - n->aggrs = (yyvsp[(6) - (9)].list); - n->groups = (yyvsp[(9) - (9)].list); + n->source = (yyvsp[-7].node); + n->columns = (yyvsp[-5].list); + n->aggrs = (yyvsp[-3].list); + n->groups = (yyvsp[0].list); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23409 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 524: -#line 340 "third_party/libpg_query/grammar/statements/select.y" - { + case 544: /* simple_select: unpivot_keyword table_ref ON target_list_opt_comma INTO NAME_P name value_or_values name_list_opt_comma_opt_bracket */ +#line 344 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (9)].node); - n->unpivots = (yyvsp[(9) - (9)].list); + n->source = (yyvsp[-7].node); + n->unpivots = (yyvsp[0].list); PGPivot *piv = makeNode(PGPivot); - piv->unpivot_columns = list_make1(makeString((yyvsp[(7) - (9)].str))); - piv->pivot_value = (yyvsp[(4) - (9)].list); + piv->unpivot_columns = list_make1(makeString((yyvsp[-2].str))); + piv->pivot_value = (yyvsp[-5].list); n->columns = list_make1(piv); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} + } +#line 23427 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 525: -#line 354 "third_party/libpg_query/grammar/statements/select.y" - { + case 545: /* simple_select: unpivot_keyword table_ref ON target_list_opt_comma */ +#line 358 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *res = makeNode(PGSelectStmt); PGPivotStmt *n = makeNode(PGPivotStmt); - n->source = (yyvsp[(2) - (4)].node); + n->source = (yyvsp[-2].node); n->unpivots = list_make1(makeString("value")); PGPivot *piv = makeNode(PGPivot); piv->unpivot_columns = list_make1(makeString("name")); - piv->pivot_value = (yyvsp[(4) - (4)].list); + piv->pivot_value = (yyvsp[0].list); n->columns = list_make1(piv); res->pivot = n; (yyval.node) = (PGNode *)res; - ;} - break; - - case 532: -#line 383 "third_party/libpg_query/grammar/statements/select.y" - { - PGPivot *n = makeNode(PGPivot); - n->pivot_columns = list_make1((yyvsp[(1) - (1)].node)); - (yyval.node) = (PGNode *) n; - ;} + } +#line 23445 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 533: -#line 389 "third_party/libpg_query/grammar/statements/select.y" - { + case 552: /* pivot_column_entry: b_expr */ +#line 387 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->pivot_columns = list_make1((yyvsp[(1) - (5)].node)); - n->subquery = (yyvsp[(4) - (5)].node); + n->pivot_columns = list_make1((yyvsp[0].node)); (yyval.node) = (PGNode *) n; - ;} + } +#line 23455 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 534: -#line 395 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 553: /* pivot_column_entry: b_expr IN_P '(' select_no_parens ')' */ +#line 393 "third_party/libpg_query/grammar/statements/select.y" + { + PGPivot *n = makeNode(PGPivot); + n->pivot_columns = list_make1((yyvsp[-4].node)); + n->subquery = (yyvsp[-1].node); + (yyval.node) = (PGNode *) n; + } +#line 23466 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 535: + case 554: /* pivot_column_entry: single_pivot_value */ #line 399 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 23472 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 536: -#line 400 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + case 555: /* pivot_column_list_internal: pivot_column_entry */ +#line 403 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 23478 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 537: + case 556: /* pivot_column_list_internal: pivot_column_list_internal ',' pivot_column_entry */ #line 404 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 23484 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 538: -#line 405 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 557: /* pivot_column_list: pivot_column_list_internal */ +#line 408 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 23490 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 539: -#line 420 "third_party/libpg_query/grammar/statements/select.y" - { + case 558: /* pivot_column_list: pivot_column_list_internal ',' */ +#line 409 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 23496 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 559: /* with_clause: WITH cte_list */ +#line 424 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.with) = makeNode(PGWithClause); - (yyval.with)->ctes = (yyvsp[(2) - (2)].list); + (yyval.with)->ctes = (yyvsp[0].list); (yyval.with)->recursive = false; - (yyval.with)->location = (yylsp[(1) - (2)]); - ;} + (yyval.with)->location = (yylsp[-1]); + } +#line 23507 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 540: -#line 427 "third_party/libpg_query/grammar/statements/select.y" - { + case 560: /* with_clause: WITH_LA cte_list */ +#line 431 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.with) = makeNode(PGWithClause); - (yyval.with)->ctes = (yyvsp[(2) - (2)].list); + (yyval.with)->ctes = (yyvsp[0].list); (yyval.with)->recursive = false; - (yyval.with)->location = (yylsp[(1) - (2)]); - ;} + (yyval.with)->location = (yylsp[-1]); + } +#line 23518 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 541: -#line 434 "third_party/libpg_query/grammar/statements/select.y" - { + case 561: /* with_clause: WITH RECURSIVE cte_list */ +#line 438 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.with) = makeNode(PGWithClause); - (yyval.with)->ctes = (yyvsp[(3) - (3)].list); + (yyval.with)->ctes = (yyvsp[0].list); (yyval.with)->recursive = true; - (yyval.with)->location = (yylsp[(1) - (3)]); - ;} + (yyval.with)->location = (yylsp[-2]); + } +#line 23529 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 542: -#line 443 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + case 562: /* cte_list: common_table_expr */ +#line 447 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 23535 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 543: -#line 444 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + case 563: /* cte_list: cte_list ',' common_table_expr */ +#line 448 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 23541 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 544: -#line 448 "third_party/libpg_query/grammar/statements/select.y" - { + case 564: /* common_table_expr: name opt_name_list AS opt_materialized '(' PreparableStmt ')' */ +#line 452 "third_party/libpg_query/grammar/statements/select.y" + { PGCommonTableExpr *n = makeNode(PGCommonTableExpr); - n->ctename = (yyvsp[(1) - (7)].str); - n->aliascolnames = (yyvsp[(2) - (7)].list); - n->ctematerialized = (yyvsp[(4) - (7)].ctematerialize); - n->ctequery = (yyvsp[(6) - (7)].node); - n->location = (yylsp[(1) - (7)]); + n->ctename = (yyvsp[-6].str); + n->aliascolnames = (yyvsp[-5].list); + n->ctematerialized = (yyvsp[-3].ctematerialize); + n->ctequery = (yyvsp[-1].node); + n->location = (yylsp[-6]); (yyval.node) = (PGNode *) n; - ;} + } +#line 23555 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 545: -#line 460 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ctematerialize) = PGCTEMaterializeAlways; ;} + case 565: /* opt_materialized: MATERIALIZED */ +#line 464 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ctematerialize) = PGCTEMaterializeAlways; } +#line 23561 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 546: -#line 461 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ctematerialize) = PGCTEMaterializeNever; ;} + case 566: /* opt_materialized: NOT MATERIALIZED */ +#line 465 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ctematerialize) = PGCTEMaterializeNever; } +#line 23567 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 547: -#line 462 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ctematerialize) = PGCTEMaterializeDefault; ;} + case 567: /* opt_materialized: %empty */ +#line 466 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ctematerialize) = PGCTEMaterializeDefault; } +#line 23573 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 548: -#line 467 "third_party/libpg_query/grammar/statements/select.y" - { + case 568: /* into_clause: INTO OptTempTableName */ +#line 471 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.into) = makeNode(PGIntoClause); - (yyval.into)->rel = (yyvsp[(2) - (2)].range); + (yyval.into)->rel = (yyvsp[0].range); (yyval.into)->colNames = NIL; (yyval.into)->options = NIL; (yyval.into)->onCommit = PG_ONCOMMIT_NOOP; (yyval.into)->viewQuery = NULL; (yyval.into)->skipData = false; - ;} + } +#line 23587 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 549: -#line 477 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.into) = NULL; ;} + case 569: /* into_clause: %empty */ +#line 481 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.into) = NULL; } +#line 23593 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 550: -#line 486 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(3) - (3)].range); + case 570: /* OptTempTableName: TEMPORARY opt_table qualified_name */ +#line 490 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23602 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 551: -#line 491 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(3) - (3)].range); + case 571: /* OptTempTableName: TEMP opt_table qualified_name */ +#line 495 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23611 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 552: -#line 496 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(4) - (4)].range); + case 572: /* OptTempTableName: LOCAL TEMPORARY opt_table qualified_name */ +#line 500 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23620 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 553: -#line 501 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(4) - (4)].range); + case 573: /* OptTempTableName: LOCAL TEMP opt_table qualified_name */ +#line 505 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23629 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 554: -#line 506 "third_party/libpg_query/grammar/statements/select.y" - { + case 574: /* OptTempTableName: GLOBAL TEMPORARY opt_table qualified_name */ +#line 510 "third_party/libpg_query/grammar/statements/select.y" + { ereport(PGWARNING, (errmsg("GLOBAL is deprecated in temporary table creation"), - parser_errposition((yylsp[(1) - (4)])))); - (yyval.range) = (yyvsp[(4) - (4)].range); + parser_errposition((yylsp[-3])))); + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23641 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 555: -#line 514 "third_party/libpg_query/grammar/statements/select.y" - { + case 575: /* OptTempTableName: GLOBAL TEMP opt_table qualified_name */ +#line 518 "third_party/libpg_query/grammar/statements/select.y" + { ereport(PGWARNING, (errmsg("GLOBAL is deprecated in temporary table creation"), - parser_errposition((yylsp[(1) - (4)])))); - (yyval.range) = (yyvsp[(4) - (4)].range); + parser_errposition((yylsp[-3])))); + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_TEMP; - ;} + } +#line 23653 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 556: -#line 522 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(3) - (3)].range); + case 576: /* OptTempTableName: UNLOGGED opt_table qualified_name */ +#line 526 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = PG_RELPERSISTENCE_UNLOGGED; - ;} + } +#line 23662 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 557: -#line 527 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(2) - (2)].range); + case 577: /* OptTempTableName: TABLE qualified_name */ +#line 531 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = RELPERSISTENCE_PERMANENT; - ;} + } +#line 23671 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 558: -#line 532 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.range) = (yyvsp[(1) - (1)].range); + case 578: /* OptTempTableName: qualified_name */ +#line 536 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.range) = (yyvsp[0].range); (yyval.range)->relpersistence = RELPERSISTENCE_PERMANENT; - ;} - break; - - case 559: -#line 538 "third_party/libpg_query/grammar/statements/select.y" - {;} + } +#line 23680 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 560: -#line 539 "third_party/libpg_query/grammar/statements/select.y" - {;} + case 579: /* opt_table: TABLE */ +#line 542 "third_party/libpg_query/grammar/statements/select.y" + {} +#line 23686 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 561: + case 580: /* opt_table: %empty */ #line 543 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + {} +#line 23692 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 562: -#line 544 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 581: /* all_or_distinct: ALL */ +#line 547 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 23698 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 563: -#line 545 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 582: /* all_or_distinct: DISTINCT */ +#line 548 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 23704 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 564: + case 583: /* all_or_distinct: %empty */ #line 549 "third_party/libpg_query/grammar/statements/select.y" - { ;} + { (yyval.boolean) = false; } +#line 23710 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 565: -#line 556 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(NIL); ;} + case 584: /* by_name: BY NAME_P */ +#line 553 "third_party/libpg_query/grammar/statements/select.y" + { } +#line 23716 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 566: -#line 557 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(4) - (5)].list); ;} + case 585: /* distinct_clause: DISTINCT */ +#line 560 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(NIL); } +#line 23722 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 567: + case 586: /* distinct_clause: DISTINCT ON '(' expr_list_opt_comma ')' */ #line 561 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL;;} + { (yyval.list) = (yyvsp[-1].list); } +#line 23728 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 568: -#line 562 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 587: /* opt_all_clause: ALL */ +#line 565 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL;} +#line 23734 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 569: + case 588: /* opt_all_clause: %empty */ #line 566 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true;;} + { (yyval.list) = NIL; } +#line 23740 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 570: -#line 567 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false;;} + case 589: /* opt_ignore_nulls: IGNORE_P NULLS_P */ +#line 570 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true;} +#line 23746 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 571: -#line 568 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 590: /* opt_ignore_nulls: RESPECT_P NULLS_P */ +#line 571 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false;} +#line 23752 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 572: + case 591: /* opt_ignore_nulls: %empty */ #line 572 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list);;} + { (yyval.boolean) = false; } +#line 23758 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 573: -#line 573 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 592: /* opt_sort_clause: sort_clause */ +#line 576 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list);} +#line 23764 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 574: + case 593: /* opt_sort_clause: %empty */ #line 577 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (3)].list); ;} + { (yyval.list) = NIL; } +#line 23770 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 575: -#line 579 "third_party/libpg_query/grammar/statements/select.y" - { + case 594: /* sort_clause: ORDER BY sortby_list */ +#line 581 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 23776 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 595: /* sort_clause: ORDER BY ALL opt_asc_desc opt_nulls_order */ +#line 583 "third_party/libpg_query/grammar/statements/select.y" + { PGSortBy *sort = makeNode(PGSortBy); PGAStar *star = makeNode(PGAStar); star->columns = true; - star->location = (yylsp[(3) - (5)]); + star->location = (yylsp[-2]); sort->node = (PGNode *) star; - sort->sortby_dir = (yyvsp[(4) - (5)].sortorder); - sort->sortby_nulls = (yyvsp[(5) - (5)].nullorder); + sort->sortby_dir = (yyvsp[-1].sortorder); + sort->sortby_nulls = (yyvsp[0].nullorder); sort->useOp = NIL; sort->location = -1; /* no operator */ (yyval.list) = list_make1(sort); - ;} + } +#line 23793 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 576: -#line 594 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].sortby)); ;} + case 596: /* sortby_list: sortby */ +#line 598 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].sortby)); } +#line 23799 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 577: -#line 595 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].sortby)); ;} + case 597: /* sortby_list: sortby_list ',' sortby */ +#line 599 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].sortby)); } +#line 23805 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 578: -#line 599 "third_party/libpg_query/grammar/statements/select.y" - { + case 598: /* sortby: a_expr USING qual_all_Op opt_nulls_order */ +#line 603 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.sortby) = makeNode(PGSortBy); - (yyval.sortby)->node = (yyvsp[(1) - (4)].node); + (yyval.sortby)->node = (yyvsp[-3].node); (yyval.sortby)->sortby_dir = SORTBY_USING; - (yyval.sortby)->sortby_nulls = (yyvsp[(4) - (4)].nullorder); - (yyval.sortby)->useOp = (yyvsp[(3) - (4)].list); - (yyval.sortby)->location = (yylsp[(3) - (4)]); - ;} + (yyval.sortby)->sortby_nulls = (yyvsp[0].nullorder); + (yyval.sortby)->useOp = (yyvsp[-1].list); + (yyval.sortby)->location = (yylsp[-1]); + } +#line 23818 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 579: -#line 608 "third_party/libpg_query/grammar/statements/select.y" - { + case 599: /* sortby: a_expr opt_asc_desc opt_nulls_order */ +#line 612 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.sortby) = makeNode(PGSortBy); - (yyval.sortby)->node = (yyvsp[(1) - (3)].node); - (yyval.sortby)->sortby_dir = (yyvsp[(2) - (3)].sortorder); - (yyval.sortby)->sortby_nulls = (yyvsp[(3) - (3)].nullorder); + (yyval.sortby)->node = (yyvsp[-2].node); + (yyval.sortby)->sortby_dir = (yyvsp[-1].sortorder); + (yyval.sortby)->sortby_nulls = (yyvsp[0].nullorder); (yyval.sortby)->useOp = NIL; (yyval.sortby)->location = -1; /* no operator */ - ;} - break; - - case 580: -#line 618 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.sortorder) = PG_SORTBY_ASC; ;} - break; - - case 581: -#line 619 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.sortorder) = PG_SORTBY_DESC; ;} + } +#line 23831 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 582: -#line 620 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.sortorder) = PG_SORTBY_DEFAULT; ;} + case 600: /* opt_asc_desc: ASC_P */ +#line 622 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.sortorder) = PG_SORTBY_ASC; } +#line 23837 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 583: + case 601: /* opt_asc_desc: DESC_P */ #line 623 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.nullorder) = PG_SORTBY_NULLS_FIRST; ;} + { (yyval.sortorder) = PG_SORTBY_DESC; } +#line 23843 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 584: + case 602: /* opt_asc_desc: %empty */ #line 624 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.nullorder) = PG_SORTBY_NULLS_LAST; ;} + { (yyval.sortorder) = PG_SORTBY_DEFAULT; } +#line 23849 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 603: /* opt_nulls_order: NULLS_LA FIRST_P */ +#line 627 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.nullorder) = PG_SORTBY_NULLS_FIRST; } +#line 23855 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 585: -#line 625 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.nullorder) = PG_SORTBY_NULLS_DEFAULT; ;} + case 604: /* opt_nulls_order: NULLS_LA LAST_P */ +#line 628 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.nullorder) = PG_SORTBY_NULLS_LAST; } +#line 23861 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 586: + case 605: /* opt_nulls_order: %empty */ #line 629 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(2) - (2)].node), (yyvsp[(1) - (2)].node)); ;} + { (yyval.nullorder) = PG_SORTBY_NULLS_DEFAULT; } +#line 23867 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 587: -#line 630 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(1) - (2)].node), (yyvsp[(2) - (2)].node)); ;} + case 606: /* select_limit: limit_clause offset_clause */ +#line 633 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2((yyvsp[0].node), (yyvsp[-1].node)); } +#line 23873 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 588: -#line 631 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2(NULL, (yyvsp[(1) - (1)].node)); ;} + case 607: /* select_limit: offset_clause limit_clause */ +#line 634 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2((yyvsp[-1].node), (yyvsp[0].node)); } +#line 23879 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 589: -#line 632 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(1) - (1)].node), NULL); ;} + case 608: /* select_limit: limit_clause */ +#line 635 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2(NULL, (yyvsp[0].node)); } +#line 23885 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 590: + case 609: /* select_limit: offset_clause */ #line 636 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = list_make2((yyvsp[0].node), NULL); } +#line 23891 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 591: -#line 637 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2(NULL,NULL); ;} + case 610: /* opt_select_limit: select_limit */ +#line 640 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 23897 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 592: -#line 642 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 611: /* opt_select_limit: %empty */ +#line 641 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2(NULL,NULL); } +#line 23903 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 593: -#line 644 "third_party/libpg_query/grammar/statements/select.y" - { + case 612: /* limit_clause: LIMIT select_limit_value */ +#line 646 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23909 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 613: /* limit_clause: LIMIT select_limit_value ',' select_offset_value */ +#line 648 "third_party/libpg_query/grammar/statements/select.y" + { /* Disabled because it was too confusing, bjm 2002-02-18 */ ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("LIMIT #,# syntax is not supported"), errhint("Use separate LIMIT and OFFSET clauses."), - parser_errposition((yylsp[(1) - (4)])))); - ;} + parser_errposition((yylsp[-3])))); + } +#line 23922 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 594: -#line 660 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(3) - (5)].node); ;} + case 614: /* limit_clause: FETCH first_or_next select_fetch_first_value row_or_rows ONLY */ +#line 664 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[-2].node); } +#line 23928 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 595: -#line 662 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeIntConst(1, -1); ;} + case 615: /* limit_clause: FETCH first_or_next row_or_rows ONLY */ +#line 666 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeIntConst(1, -1); } +#line 23934 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 596: -#line 667 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 616: /* offset_clause: OFFSET select_offset_value */ +#line 671 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 23940 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 597: -#line 670 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (3)].node); ;} + case 617: /* offset_clause: OFFSET select_fetch_first_value row_or_rows */ +#line 674 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[-1].node); } +#line 23946 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 598: -#line 678 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeFloat((yyvsp[(1) - (2)].str)), true); - ;} - break; - - case 599: + case 618: /* sample_count: FCONST '%' */ #line 682 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeInteger((yyvsp[(1) - (2)].ival)), true); - ;} + { + (yyval.node) = makeSampleSize(makeFloat((yyvsp[-1].str)), true); + } +#line 23954 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 600: + case 619: /* sample_count: ICONST '%' */ #line 686 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeFloat((yyvsp[(1) - (2)].str)), true); - ;} + { + (yyval.node) = makeSampleSize(makeInteger((yyvsp[-1].ival)), true); + } +#line 23962 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 601: + case 620: /* sample_count: FCONST PERCENT */ #line 690 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeInteger((yyvsp[(1) - (2)].ival)), true); - ;} + { + (yyval.node) = makeSampleSize(makeFloat((yyvsp[-1].str)), true); + } +#line 23970 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 602: + case 621: /* sample_count: ICONST PERCENT */ #line 694 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeInteger((yyvsp[(1) - (1)].ival)), false); - ;} + { + (yyval.node) = makeSampleSize(makeInteger((yyvsp[-1].ival)), true); + } +#line 23978 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 603: + case 622: /* sample_count: ICONST */ #line 698 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleSize(makeInteger((yyvsp[(1) - (2)].ival)), false); - ;} + { + (yyval.node) = makeSampleSize(makeInteger((yyvsp[0].ival)), false); + } +#line 23986 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 604: -#line 705 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(3) - (3)].node); - ;} + case 623: /* sample_count: ICONST ROWS */ +#line 702 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeSampleSize(makeInteger((yyvsp[-1].ival)), false); + } +#line 23994 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 605: + case 624: /* sample_clause: USING SAMPLE tablesample_entry */ #line 709 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + { + (yyval.node) = (yyvsp[0].node); + } +#line 24002 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 606: -#line 716 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 625: /* sample_clause: %empty */ +#line 713 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 24008 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 607: -#line 717 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = NULL; ;} + case 626: /* opt_sample_func: ColId */ +#line 720 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 24014 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 608: -#line 722 "third_party/libpg_query/grammar/statements/select.y" - { - int seed = (yyvsp[(5) - (5)].ival); - (yyval.node) = makeSampleOptions((yyvsp[(3) - (5)].node), (yyvsp[(1) - (5)].str), &seed, (yylsp[(1) - (5)])); - ;} + case 627: /* opt_sample_func: %empty */ +#line 721 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = NULL; } +#line 24020 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 609: -#line 727 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleOptions((yyvsp[(1) - (1)].node), NULL, NULL, (yylsp[(1) - (1)])); - ;} + case 628: /* tablesample_entry: opt_sample_func '(' sample_count ')' opt_repeatable_clause */ +#line 726 "third_party/libpg_query/grammar/statements/select.y" + { + int seed = (yyvsp[0].ival); + (yyval.node) = makeSampleOptions((yyvsp[-2].node), (yyvsp[-4].str), &seed, (yylsp[-4])); + } +#line 24029 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 610: + case 629: /* tablesample_entry: sample_count */ #line 731 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeSampleOptions((yyvsp[(1) - (4)].node), (yyvsp[(3) - (4)].str), NULL, (yylsp[(1) - (4)])); - ;} + { + (yyval.node) = makeSampleOptions((yyvsp[0].node), NULL, NULL, (yylsp[0])); + } +#line 24037 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 611: + case 630: /* tablesample_entry: sample_count '(' ColId ')' */ #line 735 "third_party/libpg_query/grammar/statements/select.y" - { - int seed = (yyvsp[(5) - (6)].ival); - (yyval.node) = makeSampleOptions((yyvsp[(1) - (6)].node), (yyvsp[(3) - (6)].str), &seed, (yylsp[(1) - (6)])); - ;} + { + (yyval.node) = makeSampleOptions((yyvsp[-3].node), (yyvsp[-1].str), NULL, (yylsp[-3])); + } +#line 24045 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 612: -#line 743 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(2) - (2)].node); - ;} + case 631: /* tablesample_entry: sample_count '(' ColId ',' ICONST ')' */ +#line 739 "third_party/libpg_query/grammar/statements/select.y" + { + int seed = (yyvsp[-1].ival); + (yyval.node) = makeSampleOptions((yyvsp[-5].node), (yyvsp[-3].str), &seed, (yylsp[-5])); + } +#line 24054 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 613: -#line 749 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 632: /* tablesample_clause: TABLESAMPLE tablesample_entry */ +#line 747 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (yyvsp[0].node); + } +#line 24062 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 614: -#line 750 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 633: /* opt_tablesample_clause: tablesample_clause */ +#line 753 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24068 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 615: -#line 755 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = (yyvsp[(3) - (4)].ival); ;} + case 634: /* opt_tablesample_clause: %empty */ +#line 754 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 24074 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 616: -#line 756 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = -1; ;} + case 635: /* opt_repeatable_clause: REPEATABLE '(' ICONST ')' */ +#line 759 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = (yyvsp[-1].ival); } +#line 24080 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 617: + case 636: /* opt_repeatable_clause: %empty */ #line 760 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.ival) = -1; } +#line 24086 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 618: -#line 762 "third_party/libpg_query/grammar/statements/select.y" - { - /* LIMIT ALL is represented as a NULL constant */ - (yyval.node) = makeNullAConst((yylsp[(1) - (1)])); - ;} + case 637: /* select_limit_value: a_expr */ +#line 764 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24092 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 619: -#line 767 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeLimitPercent((yyvsp[(1) - (2)].node)); ;} + case 638: /* select_limit_value: ALL */ +#line 766 "third_party/libpg_query/grammar/statements/select.y" + { + /* LIMIT ALL is represented as a NULL constant */ + (yyval.node) = makeNullAConst((yylsp[0])); + } +#line 24101 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 620: -#line 769 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeLimitPercent(makeFloatConst((yyvsp[(1) - (2)].str),(yylsp[(1) - (2)]))); ;} + case 639: /* select_limit_value: a_expr '%' */ +#line 771 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeLimitPercent((yyvsp[-1].node)); } +#line 24107 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 621: -#line 771 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeLimitPercent(makeIntConst((yyvsp[(1) - (2)].ival),(yylsp[(1) - (2)]))); ;} + case 640: /* select_limit_value: FCONST PERCENT */ +#line 773 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeLimitPercent(makeFloatConst((yyvsp[-1].str),(yylsp[-1]))); } +#line 24113 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 622: + case 641: /* select_limit_value: ICONST PERCENT */ #line 775 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = makeLimitPercent(makeIntConst((yyvsp[-1].ival),(yylsp[-1]))); } +#line 24119 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 623: -#line 795 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 642: /* select_offset_value: a_expr */ +#line 779 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24125 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 624: -#line 797 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + case 643: /* select_fetch_first_value: c_expr */ +#line 799 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24131 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 625: -#line 799 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = doNegate((yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + case 644: /* select_fetch_first_value: '+' I_or_F_const */ +#line 801 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[0].node), (yylsp[-1])); } +#line 24137 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 626: + case 645: /* select_fetch_first_value: '-' I_or_F_const */ #line 803 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeIntConst((yyvsp[(1) - (1)].ival),(yylsp[(1) - (1)])); ;} + { (yyval.node) = doNegate((yyvsp[0].node), (yylsp[-1])); } +#line 24143 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 627: -#line 804 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeFloatConst((yyvsp[(1) - (1)].str),(yylsp[(1) - (1)])); ;} + case 646: /* I_or_F_const: Iconst */ +#line 807 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeIntConst((yyvsp[0].ival),(yylsp[0])); } +#line 24149 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 628: + case 647: /* I_or_F_const: FCONST */ #line 808 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} - break; - - case 629: -#line 809 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} + { (yyval.node) = makeFloatConst((yyvsp[0].str),(yylsp[0])); } +#line 24155 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 630: + case 648: /* row_or_rows: ROW */ #line 812 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 24161 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 631: + case 649: /* row_or_rows: ROWS */ #line 813 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} + { (yyval.ival) = 0; } +#line 24167 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 632: -#line 838 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (3)].list); ;} + case 650: /* first_or_next: FIRST_P */ +#line 816 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; } +#line 24173 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 633: -#line 840 "third_party/libpg_query/grammar/statements/select.y" - { - PGNode *node = (PGNode *) makeGroupingSet(GROUPING_SET_ALL, NIL, (yylsp[(3) - (3)])); - (yyval.list) = list_make1(node); - ;} + case 651: /* first_or_next: NEXT */ +#line 817 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; } +#line 24179 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 652: /* group_clause: GROUP_P BY group_by_list_opt_comma */ +#line 842 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24185 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 634: + case 653: /* group_clause: GROUP_P BY ALL */ #line 844 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + { + PGNode *node = (PGNode *) makeGroupingSet(GROUPING_SET_ALL, NIL, (yylsp[0])); + (yyval.list) = list_make1(node); + } +#line 24194 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 635: + case 654: /* group_clause: %empty */ #line 848 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = NIL; } +#line 24200 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 636: -#line 849 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list),(yyvsp[(3) - (3)].node)); ;} + case 655: /* group_by_list: group_by_item */ +#line 852 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 24206 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 637: + case 656: /* group_by_list: group_by_list ',' group_by_item */ #line 853 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list),(yyvsp[0].node)); } +#line 24212 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 638: -#line 854 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 657: /* group_by_list_opt_comma: group_by_list */ +#line 857 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24218 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 639: + case 658: /* group_by_list_opt_comma: group_by_list ',' */ #line 858 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 24224 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 640: -#line 859 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 659: /* group_by_item: a_expr */ +#line 862 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24230 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 641: -#line 860 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 660: /* group_by_item: empty_grouping_set */ +#line 863 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24236 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 642: -#line 861 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 661: /* group_by_item: cube_clause */ +#line 864 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24242 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 643: -#line 862 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 662: /* group_by_item: rollup_clause */ +#line 865 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24248 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 644: -#line 867 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_EMPTY, NIL, (yylsp[(1) - (2)])); - ;} + case 663: /* group_by_item: grouping_sets_clause */ +#line 866 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24254 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 645: -#line 880 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_ROLLUP, (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + case 664: /* empty_grouping_set: '(' ')' */ +#line 871 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_EMPTY, NIL, (yylsp[-1])); + } +#line 24262 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 646: -#line 887 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_CUBE, (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + case 665: /* rollup_clause: ROLLUP '(' expr_list_opt_comma ')' */ +#line 884 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_ROLLUP, (yyvsp[-1].list), (yylsp[-3])); + } +#line 24270 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 647: -#line 894 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_SETS, (yyvsp[(4) - (5)].list), (yylsp[(1) - (5)])); - ;} + case 666: /* cube_clause: CUBE '(' expr_list_opt_comma ')' */ +#line 891 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_CUBE, (yyvsp[-1].list), (yylsp[-3])); + } +#line 24278 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 648: -#line 900 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 667: /* grouping_sets_clause: GROUPING SETS '(' group_by_list_opt_comma ')' */ +#line 898 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeGroupingSet(GROUPING_SET_SETS, (yyvsp[-1].list), (yylsp[-4])); + } +#line 24286 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 649: -#line 901 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 668: /* grouping_or_grouping_id: GROUPING */ +#line 904 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 24292 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 650: + case 669: /* grouping_or_grouping_id: GROUPING_ID */ #line 905 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + { (yyval.node) = NULL; } +#line 24298 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 651: -#line 906 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 670: /* having_clause: HAVING a_expr */ +#line 909 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24304 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 652: + case 671: /* having_clause: %empty */ #line 910 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + { (yyval.node) = NULL; } +#line 24310 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 653: -#line 911 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 672: /* qualify_clause: QUALIFY a_expr */ +#line 914 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24316 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 654: + case 673: /* qualify_clause: %empty */ #line 915 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.node) = NULL; } +#line 24322 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 655: -#line 916 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 674: /* for_locking_clause: for_locking_items */ +#line 919 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24328 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 656: + case 675: /* for_locking_clause: FOR READ_P ONLY */ #line 920 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = NIL; } +#line 24334 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 657: -#line 921 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 676: /* opt_for_locking_clause: for_locking_clause */ +#line 924 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24340 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 658: + case 677: /* opt_for_locking_clause: %empty */ #line 925 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = NIL; } +#line 24346 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 659: -#line 926 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + case 678: /* for_locking_items: for_locking_item */ +#line 929 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 24352 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 660: -#line 931 "third_party/libpg_query/grammar/statements/select.y" - { + case 679: /* for_locking_items: for_locking_items for_locking_item */ +#line 930 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 24358 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 680: /* for_locking_item: for_locking_strength locked_rels_list opt_nowait_or_skip */ +#line 935 "third_party/libpg_query/grammar/statements/select.y" + { PGLockingClause *n = makeNode(PGLockingClause); - n->lockedRels = (yyvsp[(2) - (3)].list); - n->strength = (yyvsp[(1) - (3)].lockstrength); - n->waitPolicy = (yyvsp[(3) - (3)].lockwaitpolicy); + n->lockedRels = (yyvsp[-1].list); + n->strength = (yyvsp[-2].lockstrength); + n->waitPolicy = (yyvsp[0].lockwaitpolicy); (yyval.node) = (PGNode *) n; - ;} + } +#line 24370 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 661: -#line 941 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockstrength) = LCS_FORUPDATE; ;} + case 681: /* for_locking_strength: FOR UPDATE */ +#line 945 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockstrength) = LCS_FORUPDATE; } +#line 24376 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 662: -#line 942 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockstrength) = PG_LCS_FORNOKEYUPDATE; ;} + case 682: /* for_locking_strength: FOR NO KEY UPDATE */ +#line 946 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockstrength) = PG_LCS_FORNOKEYUPDATE; } +#line 24382 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 663: -#line 943 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockstrength) = PG_LCS_FORSHARE; ;} + case 683: /* for_locking_strength: FOR SHARE */ +#line 947 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockstrength) = PG_LCS_FORSHARE; } +#line 24388 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 664: -#line 944 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockstrength) = PG_LCS_FORKEYSHARE; ;} + case 684: /* for_locking_strength: FOR KEY SHARE */ +#line 948 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockstrength) = PG_LCS_FORKEYSHARE; } +#line 24394 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 665: -#line 948 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + case 685: /* locked_rels_list: OF qualified_name_list */ +#line 952 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24400 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 666: -#line 949 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 686: /* locked_rels_list: %empty */ +#line 953 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 24406 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 667: -#line 954 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockwaitpolicy) = LockWaitError; ;} + case 687: /* opt_nowait_or_skip: NOWAIT */ +#line 958 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockwaitpolicy) = LockWaitError; } +#line 24412 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 668: -#line 955 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockwaitpolicy) = PGLockWaitSkip; ;} + case 688: /* opt_nowait_or_skip: SKIP LOCKED */ +#line 959 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockwaitpolicy) = PGLockWaitSkip; } +#line 24418 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 669: -#line 956 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.lockwaitpolicy) = PGLockWaitBlock; ;} + case 689: /* opt_nowait_or_skip: %empty */ +#line 960 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.lockwaitpolicy) = PGLockWaitBlock; } +#line 24424 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 670: -#line 966 "third_party/libpg_query/grammar/statements/select.y" - { + case 690: /* values_clause: VALUES '(' expr_list_opt_comma ')' */ +#line 970 "third_party/libpg_query/grammar/statements/select.y" + { PGSelectStmt *n = makeNode(PGSelectStmt); - n->valuesLists = list_make1((yyvsp[(3) - (4)].list)); + n->valuesLists = list_make1((yyvsp[-1].list)); (yyval.node) = (PGNode *) n; - ;} + } +#line 24434 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 671: -#line 972 "third_party/libpg_query/grammar/statements/select.y" - { - PGSelectStmt *n = (PGSelectStmt *) (yyvsp[(1) - (5)].node); - n->valuesLists = lappend(n->valuesLists, (yyvsp[(4) - (5)].list)); + case 691: /* values_clause: values_clause ',' '(' expr_list_opt_comma ')' */ +#line 976 "third_party/libpg_query/grammar/statements/select.y" + { + PGSelectStmt *n = (PGSelectStmt *) (yyvsp[-4].node); + n->valuesLists = lappend(n->valuesLists, (yyvsp[-1].list)); (yyval.node) = (PGNode *) n; - ;} - break; - - case 672: -#line 980 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + } +#line 24444 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 673: -#line 981 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (2)].node); ;} + case 692: /* values_clause_opt_comma: values_clause */ +#line 984 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 24450 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 674: -#line 994 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + case 693: /* values_clause_opt_comma: values_clause ',' */ +#line 985 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[-1].node); } +#line 24456 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 675: -#line 995 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 694: /* from_clause: FROM from_list_opt_comma */ +#line 998 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24462 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 676: + case 695: /* from_clause: %empty */ #line 999 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = NIL; } +#line 24468 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 677: -#line 1000 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + case 696: /* from_list: table_ref */ +#line 1003 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 24474 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 678: + case 697: /* from_list: from_list ',' table_ref */ #line 1004 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 24480 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 679: -#line 1005 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 698: /* from_list_opt_comma: from_list */ +#line 1008 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24486 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 680: -#line 1012 "third_party/libpg_query/grammar/statements/select.y" - { - (yyvsp[(1) - (3)].range)->alias = (yyvsp[(2) - (3)].alias); - (yyvsp[(1) - (3)].range)->sample = (yyvsp[(3) - (3)].node); - (yyval.node) = (PGNode *) (yyvsp[(1) - (3)].range); - ;} + case 699: /* from_list_opt_comma: from_list ',' */ +#line 1009 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 24492 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 681: -#line 1018 "third_party/libpg_query/grammar/statements/select.y" - { - PGRangeFunction *n = (PGRangeFunction *) (yyvsp[(1) - (3)].node); - n->alias = (PGAlias*) linitial((yyvsp[(2) - (3)].list)); - n->coldeflist = (PGList*) lsecond((yyvsp[(2) - (3)].list)); - n->sample = (yyvsp[(3) - (3)].node); + case 700: /* table_ref: relation_expr opt_alias_clause opt_tablesample_clause */ +#line 1016 "third_party/libpg_query/grammar/statements/select.y" + { + (yyvsp[-2].range)->alias = (yyvsp[-1].alias); + (yyvsp[-2].range)->sample = (yyvsp[0].node); + (yyval.node) = (PGNode *) (yyvsp[-2].range); + } +#line 24502 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 701: /* table_ref: func_table func_alias_clause opt_tablesample_clause */ +#line 1022 "third_party/libpg_query/grammar/statements/select.y" + { + PGRangeFunction *n = (PGRangeFunction *) (yyvsp[-2].node); + n->alias = (PGAlias*) linitial((yyvsp[-1].list)); + n->coldeflist = (PGList*) lsecond((yyvsp[-1].list)); + n->sample = (yyvsp[0].node); (yyval.node) = (PGNode *) n; - ;} + } +#line 24514 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 682: -#line 1026 "third_party/libpg_query/grammar/statements/select.y" - { + case 702: /* table_ref: values_clause_opt_comma alias_clause opt_tablesample_clause */ +#line 1030 "third_party/libpg_query/grammar/statements/select.y" + { PGRangeSubselect *n = makeNode(PGRangeSubselect); n->lateral = false; - n->subquery = (yyvsp[(1) - (3)].node); - n->alias = (yyvsp[(2) - (3)].alias); - n->sample = (yyvsp[(3) - (3)].node); + n->subquery = (yyvsp[-2].node); + n->alias = (yyvsp[-1].alias); + n->sample = (yyvsp[0].node); (yyval.node) = (PGNode *) n; - ;} + } +#line 24527 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 683: -#line 1035 "third_party/libpg_query/grammar/statements/select.y" - { - PGRangeFunction *n = (PGRangeFunction *) (yyvsp[(2) - (3)].node); + case 703: /* table_ref: LATERAL_P func_table func_alias_clause */ +#line 1039 "third_party/libpg_query/grammar/statements/select.y" + { + PGRangeFunction *n = (PGRangeFunction *) (yyvsp[-1].node); n->lateral = true; - n->alias = (PGAlias*) linitial((yyvsp[(3) - (3)].list)); - n->coldeflist = (PGList*) lsecond((yyvsp[(3) - (3)].list)); + n->alias = (PGAlias*) linitial((yyvsp[0].list)); + n->coldeflist = (PGList*) lsecond((yyvsp[0].list)); (yyval.node) = (PGNode *) n; - ;} + } +#line 24539 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 684: -#line 1043 "third_party/libpg_query/grammar/statements/select.y" - { + case 704: /* table_ref: select_with_parens opt_alias_clause opt_tablesample_clause */ +#line 1047 "third_party/libpg_query/grammar/statements/select.y" + { PGRangeSubselect *n = makeNode(PGRangeSubselect); n->lateral = false; - n->subquery = (yyvsp[(1) - (3)].node); - n->alias = (yyvsp[(2) - (3)].alias); - n->sample = (yyvsp[(3) - (3)].node); + n->subquery = (yyvsp[-2].node); + n->alias = (yyvsp[-1].alias); + n->sample = (yyvsp[0].node); (yyval.node) = (PGNode *) n; - ;} + } +#line 24552 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 685: -#line 1052 "third_party/libpg_query/grammar/statements/select.y" - { + case 705: /* table_ref: LATERAL_P select_with_parens opt_alias_clause */ +#line 1056 "third_party/libpg_query/grammar/statements/select.y" + { PGRangeSubselect *n = makeNode(PGRangeSubselect); n->lateral = true; - n->subquery = (yyvsp[(2) - (3)].node); - n->alias = (yyvsp[(3) - (3)].alias); + n->subquery = (yyvsp[-1].node); + n->alias = (yyvsp[0].alias); n->sample = NULL; (yyval.node) = (PGNode *) n; - ;} + } +#line 24565 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 686: -#line 1061 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) (yyvsp[(1) - (1)].jexpr); - ;} + case 706: /* table_ref: joined_table */ +#line 1065 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) (yyvsp[0].jexpr); + } +#line 24573 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 687: -#line 1065 "third_party/libpg_query/grammar/statements/select.y" - { - (yyvsp[(2) - (4)].jexpr)->alias = (yyvsp[(4) - (4)].alias); - (yyval.node) = (PGNode *) (yyvsp[(2) - (4)].jexpr); - ;} + case 707: /* table_ref: '(' joined_table ')' alias_clause */ +#line 1069 "third_party/libpg_query/grammar/statements/select.y" + { + (yyvsp[-2].jexpr)->alias = (yyvsp[0].alias); + (yyval.node) = (PGNode *) (yyvsp[-2].jexpr); + } +#line 24582 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 688: -#line 1070 "third_party/libpg_query/grammar/statements/select.y" - { + case 708: /* table_ref: table_ref PIVOT '(' target_list_opt_comma FOR pivot_value_list opt_pivot_group_by ')' opt_alias_clause */ +#line 1074 "third_party/libpg_query/grammar/statements/select.y" + { PGPivotExpr *n = makeNode(PGPivotExpr); - n->source = (yyvsp[(1) - (9)].node); - n->aggrs = (yyvsp[(4) - (9)].list); - n->pivots = (yyvsp[(6) - (9)].list); - n->groups = (yyvsp[(7) - (9)].list); - n->alias = (yyvsp[(9) - (9)].alias); + n->source = (yyvsp[-8].node); + n->aggrs = (yyvsp[-5].list); + n->pivots = (yyvsp[-3].list); + n->groups = (yyvsp[-2].list); + n->alias = (yyvsp[0].alias); (yyval.node) = (PGNode *) n; - ;} + } +#line 24596 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 689: -#line 1080 "third_party/libpg_query/grammar/statements/select.y" - { + case 709: /* table_ref: table_ref UNPIVOT opt_include_nulls '(' unpivot_header FOR unpivot_value_list ')' opt_alias_clause */ +#line 1084 "third_party/libpg_query/grammar/statements/select.y" + { PGPivotExpr *n = makeNode(PGPivotExpr); - n->source = (yyvsp[(1) - (9)].node); - n->include_nulls = (yyvsp[(3) - (9)].boolean); - n->unpivots = (yyvsp[(5) - (9)].list); - n->pivots = (yyvsp[(7) - (9)].list); - n->alias = (yyvsp[(9) - (9)].alias); + n->source = (yyvsp[-8].node); + n->include_nulls = (yyvsp[-6].boolean); + n->unpivots = (yyvsp[-4].list); + n->pivots = (yyvsp[-2].list); + n->alias = (yyvsp[0].alias); (yyval.node) = (PGNode *) n; - ;} + } +#line 24610 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 690: -#line 1092 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (3)].list); ;} + case 710: /* opt_pivot_group_by: GROUP_P BY name_list_opt_comma */ +#line 1096 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 24616 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 691: -#line 1093 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NULL; ;} + case 711: /* opt_pivot_group_by: %empty */ +#line 1097 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NULL; } +#line 24622 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 692: -#line 1096 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + case 712: /* opt_include_nulls: INCLUDE_P NULLS_P */ +#line 1100 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 24628 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 693: -#line 1097 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 713: /* opt_include_nulls: EXCLUDE NULLS_P */ +#line 1101 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 24634 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 694: -#line 1098 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 714: /* opt_include_nulls: %empty */ +#line 1102 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 24640 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 695: -#line 1102 "third_party/libpg_query/grammar/statements/select.y" - { + case 715: /* single_pivot_value: b_expr IN_P '(' target_list_opt_comma ')' */ +#line 1106 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->pivot_columns = list_make1((yyvsp[(1) - (5)].node)); - n->pivot_value = (yyvsp[(4) - (5)].list); + n->pivot_columns = list_make1((yyvsp[-4].node)); + n->pivot_value = (yyvsp[-1].list); (yyval.node) = (PGNode *) n; - ;} + } +#line 24651 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 696: -#line 1110 "third_party/libpg_query/grammar/statements/select.y" - { + case 716: /* single_pivot_value: b_expr IN_P ColIdOrString */ +#line 1114 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->pivot_columns = list_make1((yyvsp[(1) - (3)].node)); - n->pivot_enum = (yyvsp[(3) - (3)].str); + n->pivot_columns = list_make1((yyvsp[-2].node)); + n->pivot_enum = (yyvsp[0].str); (yyval.node) = (PGNode *) n; - ;} + } +#line 24662 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 697: -#line 1119 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + case 717: /* pivot_header: d_expr */ +#line 1123 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 24668 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 698: -#line 1120 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + case 718: /* pivot_header: '(' c_expr_list_opt_comma ')' */ +#line 1124 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 24674 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 699: -#line 1124 "third_party/libpg_query/grammar/statements/select.y" - { + case 719: /* pivot_value: pivot_header IN_P '(' target_list_opt_comma ')' */ +#line 1128 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->pivot_columns = (yyvsp[(1) - (5)].list); - n->pivot_value = (yyvsp[(4) - (5)].list); + n->pivot_columns = (yyvsp[-4].list); + n->pivot_value = (yyvsp[-1].list); (yyval.node) = (PGNode *) n; - ;} + } +#line 24685 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 700: -#line 1132 "third_party/libpg_query/grammar/statements/select.y" - { + case 720: /* pivot_value: pivot_header IN_P ColIdOrString */ +#line 1136 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->pivot_columns = (yyvsp[(1) - (3)].list); - n->pivot_enum = (yyvsp[(3) - (3)].str); + n->pivot_columns = (yyvsp[-2].list); + n->pivot_enum = (yyvsp[0].str); (yyval.node) = (PGNode *) n; - ;} + } +#line 24696 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 701: -#line 1141 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + case 721: /* pivot_value_list: pivot_value */ +#line 1145 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 24704 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 702: -#line 1145 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); - ;} + case 722: /* pivot_value_list: pivot_value_list pivot_value */ +#line 1149 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); + } +#line 24712 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 703: -#line 1151 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 723: /* unpivot_header: ColIdOrString */ +#line 1155 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 24718 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 704: -#line 1152 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + case 724: /* unpivot_header: '(' name_list_opt_comma ')' */ +#line 1156 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 24724 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 705: -#line 1157 "third_party/libpg_query/grammar/statements/select.y" - { + case 725: /* unpivot_value: unpivot_header IN_P '(' target_list_opt_comma ')' */ +#line 1161 "third_party/libpg_query/grammar/statements/select.y" + { PGPivot *n = makeNode(PGPivot); - n->unpivot_columns = (yyvsp[(1) - (5)].list); - n->pivot_value = (yyvsp[(4) - (5)].list); + n->unpivot_columns = (yyvsp[-4].list); + n->pivot_value = (yyvsp[-1].list); (yyval.node) = (PGNode *) n; - ;} - break; - - case 706: -#line 1166 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + } +#line 24735 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 707: + case 726: /* unpivot_value_list: unpivot_value */ #line 1170 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 24743 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 708: -#line 1195 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.jexpr) = (yyvsp[(2) - (3)].jexpr); - ;} + case 727: /* unpivot_value_list: unpivot_value_list unpivot_value */ +#line 1174 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); + } +#line 24751 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 709: + case 728: /* joined_table: '(' joined_table ')' */ #line 1199 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.jexpr) = (yyvsp[-1].jexpr); + } +#line 24759 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 729: /* joined_table: table_ref CROSS JOIN table_ref */ +#line 1203 "third_party/libpg_query/grammar/statements/select.y" + { /* CROSS JOIN is same as unqualified inner join */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_INNER; n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (4)].node); - n->rarg = (yyvsp[(4) - (4)].node); + n->larg = (yyvsp[-3].node); + n->rarg = (yyvsp[0].node); n->usingClause = NIL; n->quals = NULL; - n->location = (yylsp[(2) - (4)]); + n->location = (yylsp[-2]); (yyval.jexpr) = n; - ;} + } +#line 24776 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 710: -#line 1212 "third_party/libpg_query/grammar/statements/select.y" - { + case 730: /* joined_table: table_ref join_type JOIN table_ref join_qual */ +#line 1216 "third_party/libpg_query/grammar/statements/select.y" + { PGJoinExpr *n = makeNode(PGJoinExpr); - n->jointype = (yyvsp[(2) - (5)].jtype); + n->jointype = (yyvsp[-3].jtype); n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (5)].node); - n->rarg = (yyvsp[(4) - (5)].node); - if ((yyvsp[(5) - (5)].node) != NULL && IsA((yyvsp[(5) - (5)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(5) - (5)].node); /* USING clause */ + n->larg = (yyvsp[-4].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(5) - (5)].node); /* ON clause */ - n->location = (yylsp[(2) - (5)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-3]); (yyval.jexpr) = n; - ;} + } +#line 24794 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 711: -#line 1226 "third_party/libpg_query/grammar/statements/select.y" - { + case 731: /* joined_table: table_ref JOIN table_ref join_qual */ +#line 1230 "third_party/libpg_query/grammar/statements/select.y" + { /* letting join_type reduce to empty doesn't work */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_INNER; n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (4)].node); - n->rarg = (yyvsp[(3) - (4)].node); - if ((yyvsp[(4) - (4)].node) != NULL && IsA((yyvsp[(4) - (4)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(4) - (4)].node); /* USING clause */ + n->larg = (yyvsp[-3].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(4) - (4)].node); /* ON clause */ - n->location = (yylsp[(2) - (4)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-2]); (yyval.jexpr) = n; - ;} + } +#line 24813 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 712: -#line 1241 "third_party/libpg_query/grammar/statements/select.y" - { + case 732: /* joined_table: table_ref NATURAL join_type JOIN table_ref */ +#line 1245 "third_party/libpg_query/grammar/statements/select.y" + { PGJoinExpr *n = makeNode(PGJoinExpr); - n->jointype = (yyvsp[(3) - (5)].jtype); + n->jointype = (yyvsp[-2].jtype); n->joinreftype = PG_JOIN_NATURAL; - n->larg = (yyvsp[(1) - (5)].node); - n->rarg = (yyvsp[(5) - (5)].node); + n->larg = (yyvsp[-4].node); + n->rarg = (yyvsp[0].node); n->usingClause = NIL; /* figure out which columns later... */ n->quals = NULL; /* fill later */ - n->location = (yylsp[(2) - (5)]); + n->location = (yylsp[-3]); (yyval.jexpr) = n; - ;} + } +#line 24829 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 713: -#line 1253 "third_party/libpg_query/grammar/statements/select.y" - { + case 733: /* joined_table: table_ref NATURAL JOIN table_ref */ +#line 1257 "third_party/libpg_query/grammar/statements/select.y" + { /* letting join_type reduce to empty doesn't work */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_INNER; n->joinreftype = PG_JOIN_NATURAL; - n->larg = (yyvsp[(1) - (4)].node); - n->rarg = (yyvsp[(4) - (4)].node); + n->larg = (yyvsp[-3].node); + n->rarg = (yyvsp[0].node); n->usingClause = NIL; /* figure out which columns later... */ n->quals = NULL; /* fill later */ - n->location = (yylsp[(2) - (4)]); + n->location = (yylsp[-2]); (yyval.jexpr) = n; - ;} + } +#line 24846 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 714: -#line 1266 "third_party/libpg_query/grammar/statements/select.y" - { + case 734: /* joined_table: table_ref ASOF join_type JOIN table_ref join_qual */ +#line 1270 "third_party/libpg_query/grammar/statements/select.y" + { PGJoinExpr *n = makeNode(PGJoinExpr); - n->jointype = (yyvsp[(3) - (6)].jtype); + n->jointype = (yyvsp[-3].jtype); n->joinreftype = PG_JOIN_ASOF; - n->larg = (yyvsp[(1) - (6)].node); - n->rarg = (yyvsp[(5) - (6)].node); - if ((yyvsp[(6) - (6)].node) != NULL && IsA((yyvsp[(6) - (6)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(6) - (6)].node); /* USING clause */ + n->larg = (yyvsp[-5].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(6) - (6)].node); /* ON clause */ - n->location = (yylsp[(2) - (6)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-4]); (yyval.jexpr) = n; - ;} + } +#line 24864 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 715: -#line 1280 "third_party/libpg_query/grammar/statements/select.y" - { + case 735: /* joined_table: table_ref ASOF JOIN table_ref join_qual */ +#line 1284 "third_party/libpg_query/grammar/statements/select.y" + { PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_INNER; n->joinreftype = PG_JOIN_ASOF; - n->larg = (yyvsp[(1) - (5)].node); - n->rarg = (yyvsp[(4) - (5)].node); - if ((yyvsp[(5) - (5)].node) != NULL && IsA((yyvsp[(5) - (5)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(5) - (5)].node); /* USING clause */ + n->larg = (yyvsp[-4].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(5) - (5)].node); /* ON clause */ - n->location = (yylsp[(2) - (5)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-3]); (yyval.jexpr) = n; - ;} + } +#line 24882 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 716: -#line 1294 "third_party/libpg_query/grammar/statements/select.y" - { + case 736: /* joined_table: table_ref POSITIONAL JOIN table_ref */ +#line 1298 "third_party/libpg_query/grammar/statements/select.y" + { /* POSITIONAL JOIN is a coordinated scan */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_POSITION; n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (4)].node); - n->rarg = (yyvsp[(4) - (4)].node); + n->larg = (yyvsp[-3].node); + n->rarg = (yyvsp[0].node); n->usingClause = NIL; n->quals = NULL; - n->location = (yylsp[(2) - (4)]); + n->location = (yylsp[-2]); (yyval.jexpr) = n; - ;} + } +#line 24899 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 717: -#line 1307 "third_party/libpg_query/grammar/statements/select.y" - { + case 737: /* joined_table: table_ref ANTI JOIN table_ref join_qual */ +#line 1311 "third_party/libpg_query/grammar/statements/select.y" + { /* ANTI JOIN is a filter */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_ANTI; n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (5)].node); - n->rarg = (yyvsp[(4) - (5)].node); - if ((yyvsp[(5) - (5)].node) != NULL && IsA((yyvsp[(5) - (5)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(5) - (5)].node); /* USING clause */ + n->larg = (yyvsp[-4].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(5) - (5)].node); /* ON clause */ - n->location = (yylsp[(2) - (5)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-3]); (yyval.jexpr) = n; - ;} + } +#line 24918 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 718: -#line 1322 "third_party/libpg_query/grammar/statements/select.y" - { + case 738: /* joined_table: table_ref SEMI JOIN table_ref join_qual */ +#line 1326 "third_party/libpg_query/grammar/statements/select.y" + { /* SEMI JOIN is also a filter */ PGJoinExpr *n = makeNode(PGJoinExpr); n->jointype = PG_JOIN_SEMI; n->joinreftype = PG_JOIN_REGULAR; - n->larg = (yyvsp[(1) - (5)].node); - n->rarg = (yyvsp[(4) - (5)].node); - if ((yyvsp[(5) - (5)].node) != NULL && IsA((yyvsp[(5) - (5)].node), PGList)) - n->usingClause = (PGList *) (yyvsp[(5) - (5)].node); /* USING clause */ + n->larg = (yyvsp[-4].node); + n->rarg = (yyvsp[-1].node); + if ((yyvsp[0].node) != NULL && IsA((yyvsp[0].node), PGList)) + n->usingClause = (PGList *) (yyvsp[0].node); /* USING clause */ else - n->quals = (yyvsp[(5) - (5)].node); /* ON clause */ - n->location = (yylsp[(2) - (5)]); - n->location = (yylsp[(2) - (5)]); + n->quals = (yyvsp[0].node); /* ON clause */ + n->location = (yylsp[-3]); + n->location = (yylsp[-3]); (yyval.jexpr) = n; - ;} + } +#line 24938 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 719: -#line 1341 "third_party/libpg_query/grammar/statements/select.y" - { + case 739: /* alias_clause: AS ColIdOrString '(' name_list_opt_comma ')' */ +#line 1345 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = makeNode(PGAlias); - (yyval.alias)->aliasname = (yyvsp[(2) - (5)].str); - (yyval.alias)->colnames = (yyvsp[(4) - (5)].list); - ;} + (yyval.alias)->aliasname = (yyvsp[-3].str); + (yyval.alias)->colnames = (yyvsp[-1].list); + } +#line 24948 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 720: -#line 1347 "third_party/libpg_query/grammar/statements/select.y" - { + case 740: /* alias_clause: AS ColIdOrString */ +#line 1351 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = makeNode(PGAlias); - (yyval.alias)->aliasname = (yyvsp[(2) - (2)].str); - ;} + (yyval.alias)->aliasname = (yyvsp[0].str); + } +#line 24957 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 721: -#line 1352 "third_party/libpg_query/grammar/statements/select.y" - { + case 741: /* alias_clause: ColId '(' name_list_opt_comma ')' */ +#line 1356 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = makeNode(PGAlias); - (yyval.alias)->aliasname = (yyvsp[(1) - (4)].str); - (yyval.alias)->colnames = (yyvsp[(3) - (4)].list); - ;} + (yyval.alias)->aliasname = (yyvsp[-3].str); + (yyval.alias)->colnames = (yyvsp[-1].list); + } +#line 24967 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 722: -#line 1358 "third_party/libpg_query/grammar/statements/select.y" - { + case 742: /* alias_clause: ColId */ +#line 1362 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = makeNode(PGAlias); - (yyval.alias)->aliasname = (yyvsp[(1) - (1)].str); - ;} - break; - - case 723: -#line 1364 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.alias) = (yyvsp[(1) - (1)].alias); ;} + (yyval.alias)->aliasname = (yyvsp[0].str); + } +#line 24976 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 724: -#line 1365 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.alias) = NULL; ;} + case 743: /* opt_alias_clause: alias_clause */ +#line 1368 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = (yyvsp[0].alias); } +#line 24982 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 725: -#line 1374 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make2((yyvsp[(1) - (1)].alias), NIL); - ;} + case 744: /* opt_alias_clause: %empty */ +#line 1369 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.alias) = NULL; } +#line 24988 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 726: + case 745: /* func_alias_clause: alias_clause */ #line 1378 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make2(NULL, (yyvsp[(3) - (4)].list)); - ;} + { + (yyval.list) = list_make2((yyvsp[0].alias), NIL); + } +#line 24996 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 727: + case 746: /* func_alias_clause: AS '(' TableFuncElementList ')' */ #line 1382 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.list) = list_make2(NULL, (yyvsp[-1].list)); + } +#line 25004 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 747: /* func_alias_clause: AS ColIdOrString '(' TableFuncElementList ')' */ +#line 1386 "third_party/libpg_query/grammar/statements/select.y" + { PGAlias *a = makeNode(PGAlias); - a->aliasname = (yyvsp[(2) - (5)].str); - (yyval.list) = list_make2(a, (yyvsp[(4) - (5)].list)); - ;} + a->aliasname = (yyvsp[-3].str); + (yyval.list) = list_make2(a, (yyvsp[-1].list)); + } +#line 25014 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 728: -#line 1388 "third_party/libpg_query/grammar/statements/select.y" - { + case 748: /* func_alias_clause: ColId '(' TableFuncElementList ')' */ +#line 1392 "third_party/libpg_query/grammar/statements/select.y" + { PGAlias *a = makeNode(PGAlias); - a->aliasname = (yyvsp[(1) - (4)].str); - (yyval.list) = list_make2(a, (yyvsp[(3) - (4)].list)); - ;} + a->aliasname = (yyvsp[-3].str); + (yyval.list) = list_make2(a, (yyvsp[-1].list)); + } +#line 25024 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 729: -#line 1394 "third_party/libpg_query/grammar/statements/select.y" - { + case 749: /* func_alias_clause: %empty */ +#line 1398 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2(NULL, NIL); - ;} + } +#line 25032 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 730: -#line 1399 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_FULL; ;} + case 750: /* join_type: FULL join_outer */ +#line 1403 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_FULL; } +#line 25038 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 731: -#line 1400 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_LEFT; ;} + case 751: /* join_type: LEFT join_outer */ +#line 1404 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_LEFT; } +#line 25044 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 732: -#line 1401 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_RIGHT; ;} + case 752: /* join_type: RIGHT join_outer */ +#line 1405 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_RIGHT; } +#line 25050 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 733: -#line 1402 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_SEMI; ;} + case 753: /* join_type: SEMI */ +#line 1406 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_SEMI; } +#line 25056 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 734: -#line 1403 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_ANTI; ;} + case 754: /* join_type: ANTI */ +#line 1407 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_ANTI; } +#line 25062 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 735: -#line 1404 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.jtype) = PG_JOIN_INNER; ;} + case 755: /* join_type: INNER_P */ +#line 1408 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.jtype) = PG_JOIN_INNER; } +#line 25068 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 736: -#line 1408 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 756: /* join_outer: OUTER_P */ +#line 1412 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 25074 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 737: -#line 1409 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 757: /* join_outer: %empty */ +#line 1413 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 25080 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 738: -#line 1421 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) (yyvsp[(3) - (4)].list); ;} + case 758: /* join_qual: USING '(' name_list_opt_comma ')' */ +#line 1425 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) (yyvsp[-1].list); } +#line 25086 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 739: -#line 1422 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 759: /* join_qual: ON a_expr */ +#line 1426 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 25092 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 740: -#line 1428 "third_party/libpg_query/grammar/statements/select.y" - { + case 760: /* relation_expr: qualified_name */ +#line 1432 "third_party/libpg_query/grammar/statements/select.y" + { /* inheritance query, implicitly */ - (yyval.range) = (yyvsp[(1) - (1)].range); + (yyval.range) = (yyvsp[0].range); (yyval.range)->inh = true; (yyval.range)->alias = NULL; - ;} + } +#line 25103 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 741: -#line 1435 "third_party/libpg_query/grammar/statements/select.y" - { + case 761: /* relation_expr: qualified_name '*' */ +#line 1439 "third_party/libpg_query/grammar/statements/select.y" + { /* inheritance query, explicitly */ - (yyval.range) = (yyvsp[(1) - (2)].range); + (yyval.range) = (yyvsp[-1].range); (yyval.range)->inh = true; (yyval.range)->alias = NULL; - ;} + } +#line 25114 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 742: -#line 1442 "third_party/libpg_query/grammar/statements/select.y" - { + case 762: /* relation_expr: ONLY qualified_name */ +#line 1446 "third_party/libpg_query/grammar/statements/select.y" + { /* no inheritance */ - (yyval.range) = (yyvsp[(2) - (2)].range); + (yyval.range) = (yyvsp[0].range); (yyval.range)->inh = false; (yyval.range)->alias = NULL; - ;} + } +#line 25125 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 743: -#line 1449 "third_party/libpg_query/grammar/statements/select.y" - { + case 763: /* relation_expr: ONLY '(' qualified_name ')' */ +#line 1453 "third_party/libpg_query/grammar/statements/select.y" + { /* no inheritance, SQL99-style syntax */ - (yyval.range) = (yyvsp[(3) - (4)].range); + (yyval.range) = (yyvsp[-1].range); (yyval.range)->inh = false; (yyval.range)->alias = NULL; - ;} + } +#line 25136 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 744: -#line 1481 "third_party/libpg_query/grammar/statements/select.y" - { + case 764: /* func_table: func_expr_windowless opt_ordinality */ +#line 1485 "third_party/libpg_query/grammar/statements/select.y" + { PGRangeFunction *n = makeNode(PGRangeFunction); n->lateral = false; - n->ordinality = (yyvsp[(2) - (2)].boolean); + n->ordinality = (yyvsp[0].boolean); n->is_rowsfrom = false; - n->functions = list_make1(list_make2((yyvsp[(1) - (2)].node), NIL)); + n->functions = list_make1(list_make2((yyvsp[-1].node), NIL)); n->sample = NULL; /* alias and coldeflist are set by table_ref production */ (yyval.node) = (PGNode *) n; - ;} + } +#line 25151 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 745: -#line 1492 "third_party/libpg_query/grammar/statements/select.y" - { + case 765: /* func_table: ROWS FROM '(' rowsfrom_list ')' opt_ordinality */ +#line 1496 "third_party/libpg_query/grammar/statements/select.y" + { PGRangeFunction *n = makeNode(PGRangeFunction); n->lateral = false; - n->ordinality = (yyvsp[(6) - (6)].boolean); + n->ordinality = (yyvsp[0].boolean); n->is_rowsfrom = true; - n->functions = (yyvsp[(4) - (6)].list); + n->functions = (yyvsp[-2].list); n->sample = NULL; /* alias and coldeflist are set by table_ref production */ (yyval.node) = (PGNode *) n; - ;} - break; - - case 746: -#line 1505 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(1) - (2)].node), (yyvsp[(2) - (2)].list)); ;} + } +#line 25166 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 747: + case 766: /* rowsfrom_item: func_expr_windowless opt_col_def_list */ #line 1509 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} - break; - - case 748: -#line 1510 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} + { (yyval.list) = list_make2((yyvsp[-1].node), (yyvsp[0].list)); } +#line 25172 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 749: + case 767: /* rowsfrom_list: rowsfrom_item */ #line 1513 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + { (yyval.list) = list_make1((yyvsp[0].list)); } +#line 25178 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 750: + case 768: /* rowsfrom_list: rowsfrom_list ',' rowsfrom_item */ #line 1514 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].list)); } +#line 25184 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 751: + case 769: /* opt_col_def_list: AS '(' TableFuncElementList ')' */ #line 1517 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 25190 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 752: + case 770: /* opt_col_def_list: %empty */ #line 1518 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + { (yyval.list) = NIL; } +#line 25196 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 753: -#line 1523 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 771: /* opt_ordinality: WITH_LA ORDINALITY */ +#line 1521 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 25202 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 754: -#line 1524 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 772: /* opt_ordinality: %empty */ +#line 1522 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 25208 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 755: -#line 1530 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + case 773: /* where_clause: WHERE a_expr */ +#line 1527 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 25214 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 756: + case 774: /* where_clause: %empty */ +#line 1528 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 25220 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 775: /* TableFuncElementList: TableFuncElement */ #line 1534 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 25228 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 757: -#line 1540 "third_party/libpg_query/grammar/statements/select.y" - { + case 776: /* TableFuncElementList: TableFuncElementList ',' TableFuncElement */ +#line 1538 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 25236 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 777: /* TableFuncElement: ColIdOrString Typename opt_collate_clause */ +#line 1544 "third_party/libpg_query/grammar/statements/select.y" + { PGColumnDef *n = makeNode(PGColumnDef); - n->colname = (yyvsp[(1) - (3)].str); - n->typeName = (yyvsp[(2) - (3)].typnam); + n->colname = (yyvsp[-2].str); + n->typeName = (yyvsp[-1].typnam); n->inhcount = 0; n->is_local = true; n->is_not_null = false; @@ -24754,416 +25248,470 @@ YYLTYPE yylloc; n->storage = 0; n->raw_default = NULL; n->cooked_default = NULL; - n->collClause = (PGCollateClause *) (yyvsp[(3) - (3)].node); + n->collClause = (PGCollateClause *) (yyvsp[0].node); n->collOid = InvalidOid; n->constraints = NIL; - n->location = (yylsp[(1) - (3)]); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *)n; - ;} + } +#line 25258 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 758: -#line 1561 "third_party/libpg_query/grammar/statements/select.y" - { + case 778: /* opt_collate_clause: COLLATE any_name */ +#line 1565 "third_party/libpg_query/grammar/statements/select.y" + { PGCollateClause *n = makeNode(PGCollateClause); n->arg = NULL; - n->collname = (yyvsp[(2) - (2)].list); - n->location = (yylsp[(1) - (2)]); + n->collname = (yyvsp[0].list); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *) n; - ;} + } +#line 25270 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 759: -#line 1568 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 779: /* opt_collate_clause: %empty */ +#line 1572 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 25276 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 760: -#line 1581 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1(list_make2(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].typnam))); - ;} + case 780: /* colid_type_list: ColId Typename */ +#line 1585 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make1(list_make2(makeString((yyvsp[-1].str)), (yyvsp[0].typnam))); + } +#line 25284 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 761: -#line 1584 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (4)].list), list_make2(makeString((yyvsp[(3) - (4)].str)), (yyvsp[(4) - (4)].typnam))); - ;} + case 781: /* colid_type_list: colid_type_list ',' ColId Typename */ +#line 1588 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-3].list), list_make2(makeString((yyvsp[-1].str)), (yyvsp[0].typnam))); + } +#line 25292 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 764: -#line 1591 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 784: /* opt_Typename: Typename */ +#line 1595 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25298 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 765: -#line 1592 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = NULL; ;} + case 785: /* opt_Typename: %empty */ +#line 1596 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = NULL; } +#line 25304 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 766: -#line 1595 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (2)].typnam); - (yyval.typnam)->arrayBounds = (yyvsp[(2) - (2)].list); - ;} + case 786: /* Typename: SimpleTypename opt_array_bounds */ +#line 1599 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-1].typnam); + (yyval.typnam)->arrayBounds = (yyvsp[0].list); + } +#line 25313 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 767: -#line 1600 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(2) - (3)].typnam); - (yyval.typnam)->arrayBounds = (yyvsp[(3) - (3)].list); + case 787: /* Typename: SETOF SimpleTypename opt_array_bounds */ +#line 1604 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-1].typnam); + (yyval.typnam)->arrayBounds = (yyvsp[0].list); (yyval.typnam)->setof = true; - ;} + } +#line 25323 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 768: -#line 1607 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (5)].typnam); - (yyval.typnam)->arrayBounds = list_make1(makeInteger((yyvsp[(4) - (5)].ival))); - ;} + case 788: /* Typename: SimpleTypename ARRAY '[' Iconst ']' */ +#line 1611 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-4].typnam); + (yyval.typnam)->arrayBounds = list_make1(makeInteger((yyvsp[-1].ival))); + } +#line 25332 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 769: -#line 1612 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(2) - (6)].typnam); - (yyval.typnam)->arrayBounds = list_make1(makeInteger((yyvsp[(5) - (6)].ival))); + case 789: /* Typename: SETOF SimpleTypename ARRAY '[' Iconst ']' */ +#line 1616 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-4].typnam); + (yyval.typnam)->arrayBounds = list_make1(makeInteger((yyvsp[-1].ival))); (yyval.typnam)->setof = true; - ;} + } +#line 25342 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 770: -#line 1618 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (2)].typnam); + case 790: /* Typename: SimpleTypename ARRAY */ +#line 1622 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-1].typnam); (yyval.typnam)->arrayBounds = list_make1(makeInteger(-1)); - ;} + } +#line 25351 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 771: -#line 1623 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(2) - (3)].typnam); + case 791: /* Typename: SETOF SimpleTypename ARRAY */ +#line 1627 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-1].typnam); (yyval.typnam)->arrayBounds = list_make1(makeInteger(-1)); (yyval.typnam)->setof = true; - ;} + } +#line 25361 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 772: -#line 1629 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = makeTypeNameFromNameList((yyvsp[(1) - (1)].list)); - ;} + case 792: /* Typename: qualified_typename */ +#line 1633 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = makeTypeNameFromNameList((yyvsp[0].list)); + } +#line 25369 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 773: -#line 1633 "third_party/libpg_query/grammar/statements/select.y" - { + case 793: /* Typename: RowOrStruct '(' colid_type_list ')' opt_array_bounds */ +#line 1637 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("struct"); - (yyval.typnam)->arrayBounds = (yyvsp[(5) - (5)].list); - (yyval.typnam)->typmods = (yyvsp[(3) - (5)].list); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->arrayBounds = (yyvsp[0].list); + (yyval.typnam)->typmods = (yyvsp[-2].list); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25380 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 774: -#line 1640 "third_party/libpg_query/grammar/statements/select.y" - { + case 794: /* Typename: MAP '(' type_list ')' opt_array_bounds */ +#line 1644 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("map"); - (yyval.typnam)->arrayBounds = (yyvsp[(5) - (5)].list); - (yyval.typnam)->typmods = (yyvsp[(3) - (5)].list); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->arrayBounds = (yyvsp[0].list); + (yyval.typnam)->typmods = (yyvsp[-2].list); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25391 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 775: -#line 1647 "third_party/libpg_query/grammar/statements/select.y" - { + case 795: /* Typename: UNION '(' colid_type_list ')' opt_array_bounds */ +#line 1651 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("union"); - (yyval.typnam)->arrayBounds = (yyvsp[(5) - (5)].list); - (yyval.typnam)->typmods = (yyvsp[(3) - (5)].list); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->arrayBounds = (yyvsp[0].list); + (yyval.typnam)->typmods = (yyvsp[-2].list); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25402 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 776: -#line 1656 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2(makeString((yyvsp[(1) - (3)].str)), makeString((yyvsp[(3) - (3)].str))); ;} + case 796: /* qualified_typename: IDENT '.' IDENT */ +#line 1660 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2(makeString((yyvsp[-2].str)), makeString((yyvsp[0].str))); } +#line 25408 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 777: -#line 1657 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeString((yyvsp[(3) - (3)].str))); ;} + case 797: /* qualified_typename: qualified_typename '.' IDENT */ +#line 1661 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), makeString((yyvsp[0].str))); } +#line 25414 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 778: -#line 1662 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeInteger(-1)); ;} + case 798: /* opt_array_bounds: opt_array_bounds '[' ']' */ +#line 1666 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), makeInteger(-1)); } +#line 25420 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 779: -#line 1664 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (4)].list), makeInteger((yyvsp[(3) - (4)].ival))); ;} + case 799: /* opt_array_bounds: opt_array_bounds '[' Iconst ']' */ +#line 1668 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-3].list), makeInteger((yyvsp[-1].ival))); } +#line 25426 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 780: -#line 1666 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 800: /* opt_array_bounds: %empty */ +#line 1670 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 25432 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 781: -#line 1670 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 801: /* SimpleTypename: GenericType */ +#line 1674 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25438 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 782: -#line 1671 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 802: /* SimpleTypename: Numeric */ +#line 1675 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25444 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 783: -#line 1672 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 803: /* SimpleTypename: Bit */ +#line 1676 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25450 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 784: -#line 1673 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 804: /* SimpleTypename: Character */ +#line 1677 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25456 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 785: -#line 1674 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 805: /* SimpleTypename: ConstDatetime */ +#line 1678 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25462 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 786: -#line 1676 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (2)].typnam); - (yyval.typnam)->typmods = (yyvsp[(2) - (2)].list); - ;} + case 806: /* SimpleTypename: ConstInterval opt_interval */ +#line 1680 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-1].typnam); + (yyval.typnam)->typmods = (yyvsp[0].list); + } +#line 25471 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 787: -#line 1681 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (4)].typnam); + case 807: /* SimpleTypename: ConstInterval '(' Iconst ')' */ +#line 1685 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[-3].typnam); (yyval.typnam)->typmods = list_make2(makeIntConst(INTERVAL_FULL_RANGE, -1), - makeIntConst((yyvsp[(3) - (4)].ival), (yylsp[(3) - (4)]))); - ;} + makeIntConst((yyvsp[-1].ival), (yylsp[-1]))); + } +#line 25481 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 788: -#line 1700 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 808: /* ConstTypename: Numeric */ +#line 1704 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25487 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 789: -#line 1701 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 809: /* ConstTypename: ConstBit */ +#line 1705 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25493 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 790: -#line 1702 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 810: /* ConstTypename: ConstCharacter */ +#line 1706 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25499 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 791: -#line 1703 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.typnam) = (yyvsp[(1) - (1)].typnam); ;} + case 811: /* ConstTypename: ConstDatetime */ +#line 1707 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = (yyvsp[0].typnam); } +#line 25505 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 792: -#line 1715 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = makeTypeName((yyvsp[(1) - (2)].str)); - (yyval.typnam)->typmods = (yyvsp[(2) - (2)].list); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + case 812: /* GenericType: type_name_token opt_type_modifiers */ +#line 1719 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = makeTypeName((yyvsp[-1].str)); + (yyval.typnam)->typmods = (yyvsp[0].list); + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25515 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 793: -#line 1728 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + case 813: /* opt_type_modifiers: '(' opt_expr_list_opt_comma ')' */ +#line 1732 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 25521 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 794: -#line 1729 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 814: /* opt_type_modifiers: %empty */ +#line 1733 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 25527 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 795: -#line 1736 "third_party/libpg_query/grammar/statements/select.y" - { + case 815: /* Numeric: INT_P */ +#line 1740 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("int4"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25536 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 796: -#line 1741 "third_party/libpg_query/grammar/statements/select.y" - { + case 816: /* Numeric: INTEGER */ +#line 1745 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("int4"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25545 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 797: -#line 1746 "third_party/libpg_query/grammar/statements/select.y" - { + case 817: /* Numeric: SMALLINT */ +#line 1750 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("int2"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25554 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 798: -#line 1751 "third_party/libpg_query/grammar/statements/select.y" - { + case 818: /* Numeric: BIGINT */ +#line 1755 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("int8"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25563 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 799: -#line 1756 "third_party/libpg_query/grammar/statements/select.y" - { + case 819: /* Numeric: REAL */ +#line 1760 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("float4"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25572 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 800: -#line 1761 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(2) - (2)].typnam); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + case 820: /* Numeric: FLOAT_P opt_float */ +#line 1765 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[0].typnam); + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25581 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 801: -#line 1766 "third_party/libpg_query/grammar/statements/select.y" - { + case 821: /* Numeric: DOUBLE_P PRECISION */ +#line 1770 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("float8"); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25590 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 802: -#line 1771 "third_party/libpg_query/grammar/statements/select.y" - { + case 822: /* Numeric: DECIMAL_P opt_type_modifiers */ +#line 1775 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("numeric"); - (yyval.typnam)->typmods = (yyvsp[(2) - (2)].list); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->typmods = (yyvsp[0].list); + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25600 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 803: -#line 1777 "third_party/libpg_query/grammar/statements/select.y" - { + case 823: /* Numeric: DEC opt_type_modifiers */ +#line 1781 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("numeric"); - (yyval.typnam)->typmods = (yyvsp[(2) - (2)].list); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->typmods = (yyvsp[0].list); + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25610 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 804: -#line 1783 "third_party/libpg_query/grammar/statements/select.y" - { + case 824: /* Numeric: NUMERIC opt_type_modifiers */ +#line 1787 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("numeric"); - (yyval.typnam)->typmods = (yyvsp[(2) - (2)].list); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->typmods = (yyvsp[0].list); + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25620 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 805: -#line 1789 "third_party/libpg_query/grammar/statements/select.y" - { + case 825: /* Numeric: BOOLEAN_P */ +#line 1793 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("bool"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25629 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 806: -#line 1796 "third_party/libpg_query/grammar/statements/select.y" - { + case 826: /* opt_float: '(' Iconst ')' */ +#line 1800 "third_party/libpg_query/grammar/statements/select.y" + { /* * Check FLOAT() precision limits assuming IEEE floating * types - thomas 1997-09-18 */ - if ((yyvsp[(2) - (3)].ival) < 1) + if ((yyvsp[-1].ival) < 1) ereport(ERROR, (errcode(PG_ERRCODE_INVALID_PARAMETER_VALUE), errmsg("precision for type float must be at least 1 bit"), - parser_errposition((yylsp[(2) - (3)])))); - else if ((yyvsp[(2) - (3)].ival) <= 24) + parser_errposition((yylsp[-1])))); + else if ((yyvsp[-1].ival) <= 24) (yyval.typnam) = SystemTypeName("float4"); - else if ((yyvsp[(2) - (3)].ival) <= 53) + else if ((yyvsp[-1].ival) <= 53) (yyval.typnam) = SystemTypeName("float8"); else ereport(ERROR, (errcode(PG_ERRCODE_INVALID_PARAMETER_VALUE), errmsg("precision for type float must be less than 54 bits"), - parser_errposition((yylsp[(2) - (3)])))); - ;} + parser_errposition((yylsp[-1])))); + } +#line 25654 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 807: -#line 1817 "third_party/libpg_query/grammar/statements/select.y" - { + case 827: /* opt_float: %empty */ +#line 1821 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("float4"); - ;} - break; - - case 808: -#line 1827 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + } +#line 25662 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 809: + case 828: /* Bit: BitWithLength */ #line 1831 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25670 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 810: -#line 1839 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + case 829: /* Bit: BitWithoutLength */ +#line 1835 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25678 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 811: + case 830: /* ConstBit: BitWithLength */ #line 1843 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25686 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 831: /* ConstBit: BitWithoutLength */ +#line 1847 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[0].typnam); (yyval.typnam)->typmods = NIL; - ;} + } +#line 25695 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 812: -#line 1851 "third_party/libpg_query/grammar/statements/select.y" - { + case 832: /* BitWithLength: BIT opt_varying '(' expr_list_opt_comma ')' */ +#line 1855 "third_party/libpg_query/grammar/statements/select.y" + { const char *typname; - typname = (yyvsp[(2) - (5)].boolean) ? "varbit" : "bit"; + typname = (yyvsp[-3].boolean) ? "varbit" : "bit"; (yyval.typnam) = SystemTypeName(typname); - (yyval.typnam)->typmods = (yyvsp[(4) - (5)].list); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->typmods = (yyvsp[-1].list); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25708 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 813: -#line 1863 "third_party/libpg_query/grammar/statements/select.y" - { + case 833: /* BitWithoutLength: BIT opt_varying */ +#line 1867 "third_party/libpg_query/grammar/statements/select.y" + { /* bit defaults to bit(1), varbit to no limit */ - if ((yyvsp[(2) - (2)].boolean)) + if ((yyvsp[0].boolean)) { (yyval.typnam) = SystemTypeName("varbit"); } @@ -25172,873 +25720,982 @@ YYLTYPE yylloc; (yyval.typnam) = SystemTypeName("bit"); (yyval.typnam)->typmods = list_make1(makeIntConst(1, -1)); } - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} - break; - - case 814: -#line 1884 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25726 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 815: + case 834: /* Character: CharacterWithLength */ #line 1888 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25734 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 816: -#line 1894 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - ;} + case 835: /* Character: CharacterWithoutLength */ +#line 1892 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25742 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 817: + case 836: /* ConstCharacter: CharacterWithLength */ #line 1898 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.typnam) = (yyvsp[0].typnam); + } +#line 25750 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 837: /* ConstCharacter: CharacterWithoutLength */ +#line 1902 "third_party/libpg_query/grammar/statements/select.y" + { /* Length was not specified so allow to be unrestricted. * This handles problems with fixed-length (bpchar) strings * which in column definitions must default to a length * of one, but should not be constrained if the length - * was not specified. - */ - (yyval.typnam) = (yyvsp[(1) - (1)].typnam); - (yyval.typnam)->typmods = NIL; - ;} - break; - - case 818: -#line 1911 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = SystemTypeName((yyvsp[(1) - (4)].conststr)); - (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[(3) - (4)].ival), (yylsp[(3) - (4)]))); - (yyval.typnam)->location = (yylsp[(1) - (4)]); - ;} - break; - - case 819: -#line 1919 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.typnam) = SystemTypeName((yyvsp[(1) - (1)].conststr)); - /* char defaults to char(1), varchar to no limit */ - if (strcmp((yyvsp[(1) - (1)].conststr), "bpchar") == 0) - (yyval.typnam)->typmods = list_make1(makeIntConst(1, -1)); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} + * was not specified. + */ + (yyval.typnam) = (yyvsp[0].typnam); + (yyval.typnam)->typmods = NIL; + } +#line 25765 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 820: -#line 1929 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = (yyvsp[(2) - (2)].boolean) ? "varchar": "bpchar"; ;} + case 838: /* CharacterWithLength: character '(' Iconst ')' */ +#line 1915 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = SystemTypeName((yyvsp[-3].conststr)); + (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[-1].ival), (yylsp[-1]))); + (yyval.typnam)->location = (yylsp[-3]); + } +#line 25775 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 821: -#line 1931 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = (yyvsp[(2) - (2)].boolean) ? "varchar": "bpchar"; ;} + case 839: /* CharacterWithoutLength: character */ +#line 1923 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.typnam) = SystemTypeName((yyvsp[0].conststr)); + /* char defaults to char(1), varchar to no limit */ + if (strcmp((yyvsp[0].conststr), "bpchar") == 0) + (yyval.typnam)->typmods = list_make1(makeIntConst(1, -1)); + (yyval.typnam)->location = (yylsp[0]); + } +#line 25787 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 822: + case 840: /* character: CHARACTER opt_varying */ #line 1933 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "varchar"; ;} + { (yyval.conststr) = (yyvsp[0].boolean) ? "varchar": "bpchar"; } +#line 25793 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 823: + case 841: /* character: CHAR_P opt_varying */ #line 1935 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = (yyvsp[(3) - (3)].boolean) ? "varchar": "bpchar"; ;} + { (yyval.conststr) = (yyvsp[0].boolean) ? "varchar": "bpchar"; } +#line 25799 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 824: + case 842: /* character: VARCHAR */ #line 1937 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = (yyvsp[(3) - (3)].boolean) ? "varchar": "bpchar"; ;} + { (yyval.conststr) = "varchar"; } +#line 25805 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 825: + case 843: /* character: NATIONAL CHARACTER opt_varying */ #line 1939 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = (yyvsp[(2) - (2)].boolean) ? "varchar": "bpchar"; ;} + { (yyval.conststr) = (yyvsp[0].boolean) ? "varchar": "bpchar"; } +#line 25811 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 844: /* character: NATIONAL CHAR_P opt_varying */ +#line 1941 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.conststr) = (yyvsp[0].boolean) ? "varchar": "bpchar"; } +#line 25817 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 826: + case 845: /* character: NCHAR opt_varying */ #line 1943 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + { (yyval.conststr) = (yyvsp[0].boolean) ? "varchar": "bpchar"; } +#line 25823 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 827: -#line 1944 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 846: /* opt_varying: VARYING */ +#line 1947 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 25829 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 828: -#line 1952 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(5) - (5)].boolean)) + case 847: /* opt_varying: %empty */ +#line 1948 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 25835 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 848: /* ConstDatetime: TIMESTAMP '(' Iconst ')' opt_timezone */ +#line 1956 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].boolean)) (yyval.typnam) = SystemTypeName("timestamptz"); else (yyval.typnam) = SystemTypeName("timestamp"); - (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[(3) - (5)].ival), (yylsp[(3) - (5)]))); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[-2].ival), (yylsp[-2]))); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25848 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 829: -#line 1961 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(2) - (2)].boolean)) + case 849: /* ConstDatetime: TIMESTAMP opt_timezone */ +#line 1965 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].boolean)) (yyval.typnam) = SystemTypeName("timestamptz"); else (yyval.typnam) = SystemTypeName("timestamp"); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25860 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 830: -#line 1969 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(5) - (5)].boolean)) + case 850: /* ConstDatetime: TIME '(' Iconst ')' opt_timezone */ +#line 1973 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].boolean)) (yyval.typnam) = SystemTypeName("timetz"); else (yyval.typnam) = SystemTypeName("time"); - (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[(3) - (5)].ival), (yylsp[(3) - (5)]))); - (yyval.typnam)->location = (yylsp[(1) - (5)]); - ;} + (yyval.typnam)->typmods = list_make1(makeIntConst((yyvsp[-2].ival), (yylsp[-2]))); + (yyval.typnam)->location = (yylsp[-4]); + } +#line 25873 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 831: -#line 1978 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(2) - (2)].boolean)) + case 851: /* ConstDatetime: TIME opt_timezone */ +#line 1982 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].boolean)) (yyval.typnam) = SystemTypeName("timetz"); else (yyval.typnam) = SystemTypeName("time"); - (yyval.typnam)->location = (yylsp[(1) - (2)]); - ;} + (yyval.typnam)->location = (yylsp[-1]); + } +#line 25885 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 832: -#line 1989 "third_party/libpg_query/grammar/statements/select.y" - { + case 852: /* ConstInterval: INTERVAL */ +#line 1993 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.typnam) = SystemTypeName("interval"); - (yyval.typnam)->location = (yylsp[(1) - (1)]); - ;} - break; - - case 833: -#line 1996 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + (yyval.typnam)->location = (yylsp[0]); + } +#line 25894 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 834: -#line 1997 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 853: /* opt_timezone: WITH_LA TIME ZONE */ +#line 2000 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 25900 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 835: -#line 1998 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 854: /* opt_timezone: WITHOUT TIME ZONE */ +#line 2001 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 25906 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 860: -#line 2039 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(YEAR), (yylsp[(1) - (1)]))); ;} + case 855: /* opt_timezone: %empty */ +#line 2002 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 25912 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 861: -#line 2041 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MONTH), (yylsp[(1) - (1)]))); ;} - break; - - case 862: + case 880: /* opt_interval: year_keyword */ #line 2043 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DAY), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(YEAR), (yylsp[0]))); } +#line 25918 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 863: + case 881: /* opt_interval: month_keyword */ #line 2045 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(HOUR), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MONTH), (yylsp[0]))); } +#line 25924 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 864: + case 882: /* opt_interval: day_keyword */ #line 2047 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MINUTE), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DAY), (yylsp[0]))); } +#line 25930 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 865: + case 883: /* opt_interval: hour_keyword */ #line 2049 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(SECOND), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(HOUR), (yylsp[0]))); } +#line 25936 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 866: + case 884: /* opt_interval: minute_keyword */ #line 2051 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MILLISECOND), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MINUTE), (yylsp[0]))); } +#line 25942 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 867: + case 885: /* opt_interval: second_keyword */ #line 2053 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MICROSECOND), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(SECOND), (yylsp[0]))); } +#line 25948 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 868: + case 886: /* opt_interval: millisecond_keyword */ #line 2055 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(WEEK), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MILLISECOND), (yylsp[0]))); } +#line 25954 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 869: + case 887: /* opt_interval: microsecond_keyword */ #line 2057 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DECADE), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MICROSECOND), (yylsp[0]))); } +#line 25960 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 870: + case 888: /* opt_interval: week_keyword */ #line 2059 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(CENTURY), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(WEEK), (yylsp[0]))); } +#line 25966 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 871: + case 889: /* opt_interval: decade_keyword */ #line 2061 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MILLENNIUM), (yylsp[(1) - (1)]))); ;} + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DECADE), (yylsp[0]))); } +#line 25972 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 872: + case 890: /* opt_interval: century_keyword */ #line 2063 "third_party/libpg_query/grammar/statements/select.y" - { + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(CENTURY), (yylsp[0]))); } +#line 25978 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 891: /* opt_interval: millennium_keyword */ +#line 2065 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MILLENNIUM), (yylsp[0]))); } +#line 25984 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 892: /* opt_interval: year_keyword TO month_keyword */ +#line 2067 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(YEAR) | - INTERVAL_MASK(MONTH), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(MONTH), (yylsp[-2]))); + } +#line 25993 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 873: -#line 2068 "third_party/libpg_query/grammar/statements/select.y" - { + case 893: /* opt_interval: day_keyword TO hour_keyword */ +#line 2072 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DAY) | - INTERVAL_MASK(HOUR), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(HOUR), (yylsp[-2]))); + } +#line 26002 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 874: -#line 2073 "third_party/libpg_query/grammar/statements/select.y" - { + case 894: /* opt_interval: day_keyword TO minute_keyword */ +#line 2077 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DAY) | INTERVAL_MASK(HOUR) | - INTERVAL_MASK(MINUTE), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(MINUTE), (yylsp[-2]))); + } +#line 26012 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 875: -#line 2079 "third_party/libpg_query/grammar/statements/select.y" - { + case 895: /* opt_interval: day_keyword TO second_keyword */ +#line 2083 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(DAY) | INTERVAL_MASK(HOUR) | INTERVAL_MASK(MINUTE) | - INTERVAL_MASK(SECOND), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(SECOND), (yylsp[-2]))); + } +#line 26023 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 876: -#line 2086 "third_party/libpg_query/grammar/statements/select.y" - { + case 896: /* opt_interval: hour_keyword TO minute_keyword */ +#line 2090 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(HOUR) | - INTERVAL_MASK(MINUTE), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(MINUTE), (yylsp[-2]))); + } +#line 26032 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 877: -#line 2091 "third_party/libpg_query/grammar/statements/select.y" - { + case 897: /* opt_interval: hour_keyword TO second_keyword */ +#line 2095 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(HOUR) | INTERVAL_MASK(MINUTE) | - INTERVAL_MASK(SECOND), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(SECOND), (yylsp[-2]))); + } +#line 26042 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 878: -#line 2097 "third_party/libpg_query/grammar/statements/select.y" - { + case 898: /* opt_interval: minute_keyword TO second_keyword */ +#line 2101 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeIntConst(INTERVAL_MASK(MINUTE) | - INTERVAL_MASK(SECOND), (yylsp[(1) - (3)]))); - ;} + INTERVAL_MASK(SECOND), (yylsp[-2]))); + } +#line 26051 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 879: -#line 2102 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 899: /* opt_interval: %empty */ +#line 2106 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 26057 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 880: -#line 2133 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 900: /* a_expr: c_expr */ +#line 2137 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 26063 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 881: -#line 2136 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeTypeCast((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].typnam), 0, (yylsp[(2) - (3)])); ;} + case 901: /* a_expr: a_expr TYPECAST Typename */ +#line 2140 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeTypeCast((yyvsp[-2].node), (yyvsp[0].typnam), 0, (yylsp[-1])); } +#line 26069 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 882: -#line 2138 "third_party/libpg_query/grammar/statements/select.y" - { + case 902: /* a_expr: a_expr COLLATE any_name */ +#line 2142 "third_party/libpg_query/grammar/statements/select.y" + { PGCollateClause *n = makeNode(PGCollateClause); - n->arg = (yyvsp[(1) - (3)].node); - n->collname = (yyvsp[(3) - (3)].list); - n->location = (yylsp[(2) - (3)]); + n->arg = (yyvsp[-2].node); + n->collname = (yyvsp[0].list); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *) n; - ;} + } +#line 26081 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 883: -#line 2146 "third_party/libpg_query/grammar/statements/select.y" - { + case 903: /* a_expr: a_expr AT TIME ZONE a_expr */ +#line 2150 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("timezone"), - list_make2((yyvsp[(5) - (5)].node), (yyvsp[(1) - (5)].node)), - (yylsp[(2) - (5)])); - ;} - break; - - case 884: -#line 2161 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} - break; - - case 885: -#line 2163 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = doNegate((yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + list_make2((yyvsp[0].node), (yyvsp[-4].node)), + (yylsp[-3])); + } +#line 26091 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 886: + case 904: /* a_expr: '+' a_expr */ #line 2165 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[0].node), (yylsp[-1])); } +#line 26097 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 887: + case 905: /* a_expr: '-' a_expr */ #line 2167 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "-", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = doNegate((yyvsp[0].node), (yylsp[-1])); } +#line 26103 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 888: + case 906: /* a_expr: a_expr '+' a_expr */ #line 2169 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "*", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26109 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 889: + case 907: /* a_expr: a_expr '-' a_expr */ #line 2171 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "/", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "-", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26115 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 890: + case 908: /* a_expr: a_expr '*' a_expr */ #line 2173 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "//", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "*", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26121 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 891: + case 909: /* a_expr: a_expr '/' a_expr */ #line 2175 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "%", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "/", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26127 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 892: + case 910: /* a_expr: a_expr INTEGER_DIVISION a_expr */ #line 2177 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "^", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "//", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26133 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 893: + case 911: /* a_expr: a_expr '%' a_expr */ #line 2179 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "**", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "%", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26139 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 894: + case 912: /* a_expr: a_expr '^' a_expr */ #line 2181 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "^", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26145 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 895: + case 913: /* a_expr: a_expr POWER_OF a_expr */ #line 2183 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "**", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26151 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 896: + case 914: /* a_expr: a_expr '<' a_expr */ #line 2185 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26157 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 897: + case 915: /* a_expr: a_expr '>' a_expr */ #line 2187 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26163 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 898: + case 916: /* a_expr: a_expr '=' a_expr */ #line 2189 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26169 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 899: + case 917: /* a_expr: a_expr LESS_EQUALS a_expr */ #line 2191 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<>", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26175 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 900: -#line 2194 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(2) - (3)].list), (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + case 918: /* a_expr: a_expr GREATER_EQUALS a_expr */ +#line 2193 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26181 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 901: -#line 2196 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(1) - (2)].list), NULL, (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + case 919: /* a_expr: a_expr NOT_EQUALS a_expr */ +#line 2195 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<>", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26187 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 902: + case 920: /* a_expr: a_expr qual_Op a_expr */ #line 2198 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(2) - (2)].list), (yyvsp[(1) - (2)].node), NULL, (yylsp[(2) - (2)])); ;} + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[-1].list), (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26193 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 903: -#line 2201 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeAndExpr((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + case 921: /* a_expr: qual_Op a_expr */ +#line 2200 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[-1].list), NULL, (yyvsp[0].node), (yylsp[-1])); } +#line 26199 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 904: -#line 2203 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeOrExpr((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + case 922: /* a_expr: a_expr qual_Op */ +#line 2202 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[0].list), (yyvsp[-1].node), NULL, (yylsp[0])); } +#line 26205 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 905: + case 923: /* a_expr: a_expr AND a_expr */ #line 2205 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeNotExpr((yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + { (yyval.node) = makeAndExpr((yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26211 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 906: + case 924: /* a_expr: a_expr OR a_expr */ #line 2207 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeNotExpr((yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + { (yyval.node) = makeOrExpr((yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26217 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 907: + case 925: /* a_expr: NOT a_expr */ #line 2209 "third_party/libpg_query/grammar/statements/select.y" - { + { (yyval.node) = makeNotExpr((yyvsp[0].node), (yylsp[-1])); } +#line 26223 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 926: /* a_expr: NOT_LA a_expr */ +#line 2211 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeNotExpr((yyvsp[0].node), (yylsp[-1])); } +#line 26229 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 927: /* a_expr: a_expr GLOB a_expr */ +#line 2213 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_GLOB, "~~~", - (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); - ;} + (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); + } +#line 26238 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 908: -#line 2214 "third_party/libpg_query/grammar/statements/select.y" - { + case 928: /* a_expr: a_expr LIKE a_expr */ +#line 2218 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_LIKE, "~~", - (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); - ;} + (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); + } +#line 26247 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 909: -#line 2219 "third_party/libpg_query/grammar/statements/select.y" - { + case 929: /* a_expr: a_expr LIKE a_expr ESCAPE a_expr */ +#line 2223 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("like_escape"), - list_make3((yyvsp[(1) - (5)].node), (yyvsp[(3) - (5)].node), (yyvsp[(5) - (5)].node)), - (yylsp[(2) - (5)])); + list_make3((yyvsp[-4].node), (yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-3])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26258 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 910: -#line 2226 "third_party/libpg_query/grammar/statements/select.y" - { + case 930: /* a_expr: a_expr NOT_LA LIKE a_expr */ +#line 2230 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_LIKE, "!~~", - (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node), (yylsp[(2) - (4)])); - ;} + (yyvsp[-3].node), (yyvsp[0].node), (yylsp[-2])); + } +#line 26267 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 911: -#line 2231 "third_party/libpg_query/grammar/statements/select.y" - { + case 931: /* a_expr: a_expr NOT_LA LIKE a_expr ESCAPE a_expr */ +#line 2235 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("not_like_escape"), - list_make3((yyvsp[(1) - (6)].node), (yyvsp[(4) - (6)].node), (yyvsp[(6) - (6)].node)), - (yylsp[(2) - (6)])); + list_make3((yyvsp[-5].node), (yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-4])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26278 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 912: -#line 2238 "third_party/libpg_query/grammar/statements/select.y" - { + case 932: /* a_expr: a_expr ILIKE a_expr */ +#line 2242 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_ILIKE, "~~*", - (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); - ;} + (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); + } +#line 26287 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 913: -#line 2243 "third_party/libpg_query/grammar/statements/select.y" - { + case 933: /* a_expr: a_expr ILIKE a_expr ESCAPE a_expr */ +#line 2247 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("ilike_escape"), - list_make3((yyvsp[(1) - (5)].node), (yyvsp[(3) - (5)].node), (yyvsp[(5) - (5)].node)), - (yylsp[(2) - (5)])); + list_make3((yyvsp[-4].node), (yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-3])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26298 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 914: -#line 2250 "third_party/libpg_query/grammar/statements/select.y" - { + case 934: /* a_expr: a_expr NOT_LA ILIKE a_expr */ +#line 2254 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_ILIKE, "!~~*", - (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node), (yylsp[(2) - (4)])); - ;} + (yyvsp[-3].node), (yyvsp[0].node), (yylsp[-2])); + } +#line 26307 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 915: -#line 2255 "third_party/libpg_query/grammar/statements/select.y" - { + case 935: /* a_expr: a_expr NOT_LA ILIKE a_expr ESCAPE a_expr */ +#line 2259 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("not_ilike_escape"), - list_make3((yyvsp[(1) - (6)].node), (yyvsp[(4) - (6)].node), (yyvsp[(6) - (6)].node)), - (yylsp[(2) - (6)])); + list_make3((yyvsp[-5].node), (yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-4])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26318 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 916: -#line 2263 "third_party/libpg_query/grammar/statements/select.y" - { + case 936: /* a_expr: a_expr SIMILAR TO a_expr */ +#line 2267 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("similar_escape"), - list_make2((yyvsp[(4) - (4)].node), makeNullAConst(-1)), - (yylsp[(2) - (4)])); + list_make2((yyvsp[0].node), makeNullAConst(-1)), + (yylsp[-2])); (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_SIMILAR, "~", - (yyvsp[(1) - (4)].node), (PGNode *) n, (yylsp[(2) - (4)])); - ;} + (yyvsp[-3].node), (PGNode *) n, (yylsp[-2])); + } +#line 26330 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 917: -#line 2271 "third_party/libpg_query/grammar/statements/select.y" - { + case 937: /* a_expr: a_expr SIMILAR TO a_expr ESCAPE a_expr */ +#line 2275 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("similar_escape"), - list_make2((yyvsp[(4) - (6)].node), (yyvsp[(6) - (6)].node)), - (yylsp[(2) - (6)])); + list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-4])); (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_SIMILAR, "~", - (yyvsp[(1) - (6)].node), (PGNode *) n, (yylsp[(2) - (6)])); - ;} + (yyvsp[-5].node), (PGNode *) n, (yylsp[-4])); + } +#line 26342 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 918: -#line 2279 "third_party/libpg_query/grammar/statements/select.y" - { + case 938: /* a_expr: a_expr NOT_LA SIMILAR TO a_expr */ +#line 2283 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("similar_escape"), - list_make2((yyvsp[(5) - (5)].node), makeNullAConst(-1)), - (yylsp[(2) - (5)])); + list_make2((yyvsp[0].node), makeNullAConst(-1)), + (yylsp[-3])); (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_SIMILAR, "!~", - (yyvsp[(1) - (5)].node), (PGNode *) n, (yylsp[(2) - (5)])); - ;} + (yyvsp[-4].node), (PGNode *) n, (yylsp[-3])); + } +#line 26354 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 919: -#line 2287 "third_party/libpg_query/grammar/statements/select.y" - { + case 939: /* a_expr: a_expr NOT_LA SIMILAR TO a_expr ESCAPE a_expr */ +#line 2291 "third_party/libpg_query/grammar/statements/select.y" + { PGFuncCall *n = makeFuncCall(SystemFuncName("similar_escape"), - list_make2((yyvsp[(5) - (7)].node), (yyvsp[(7) - (7)].node)), - (yylsp[(2) - (7)])); + list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-5])); (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_SIMILAR, "!~", - (yyvsp[(1) - (7)].node), (PGNode *) n, (yylsp[(2) - (7)])); - ;} + (yyvsp[-6].node), (PGNode *) n, (yylsp[-5])); + } +#line 26366 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 920: -#line 2305 "third_party/libpg_query/grammar/statements/select.y" - { + case 940: /* a_expr: a_expr IS NULL_P */ +#line 2309 "third_party/libpg_query/grammar/statements/select.y" + { PGNullTest *n = makeNode(PGNullTest); - n->arg = (PGExpr *) (yyvsp[(1) - (3)].node); + n->arg = (PGExpr *) (yyvsp[-2].node); n->nulltesttype = PG_IS_NULL; - n->location = (yylsp[(2) - (3)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26378 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 921: -#line 2313 "third_party/libpg_query/grammar/statements/select.y" - { + case 941: /* a_expr: a_expr ISNULL */ +#line 2317 "third_party/libpg_query/grammar/statements/select.y" + { PGNullTest *n = makeNode(PGNullTest); - n->arg = (PGExpr *) (yyvsp[(1) - (2)].node); + n->arg = (PGExpr *) (yyvsp[-1].node); n->nulltesttype = PG_IS_NULL; - n->location = (yylsp[(2) - (2)]); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26390 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 922: -#line 2321 "third_party/libpg_query/grammar/statements/select.y" - { + case 942: /* a_expr: a_expr IS NOT NULL_P */ +#line 2325 "third_party/libpg_query/grammar/statements/select.y" + { PGNullTest *n = makeNode(PGNullTest); - n->arg = (PGExpr *) (yyvsp[(1) - (4)].node); + n->arg = (PGExpr *) (yyvsp[-3].node); n->nulltesttype = IS_NOT_NULL; - n->location = (yylsp[(2) - (4)]); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26402 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 923: -#line 2329 "third_party/libpg_query/grammar/statements/select.y" - { + case 943: /* a_expr: a_expr NOT NULL_P */ +#line 2333 "third_party/libpg_query/grammar/statements/select.y" + { PGNullTest *n = makeNode(PGNullTest); - n->arg = (PGExpr *) (yyvsp[(1) - (3)].node); + n->arg = (PGExpr *) (yyvsp[-2].node); n->nulltesttype = IS_NOT_NULL; - n->location = (yylsp[(2) - (3)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26414 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 924: -#line 2337 "third_party/libpg_query/grammar/statements/select.y" - { + case 944: /* a_expr: a_expr NOTNULL */ +#line 2341 "third_party/libpg_query/grammar/statements/select.y" + { PGNullTest *n = makeNode(PGNullTest); - n->arg = (PGExpr *) (yyvsp[(1) - (2)].node); + n->arg = (PGExpr *) (yyvsp[-1].node); n->nulltesttype = IS_NOT_NULL; - n->location = (yylsp[(2) - (2)]); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26426 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 925: -#line 2345 "third_party/libpg_query/grammar/statements/select.y" - { + case 945: /* a_expr: a_expr LAMBDA_ARROW a_expr */ +#line 2349 "third_party/libpg_query/grammar/statements/select.y" + { PGLambdaFunction *n = makeNode(PGLambdaFunction); - n->lhs = (yyvsp[(1) - (3)].node); - n->rhs = (yyvsp[(3) - (3)].node); - n->location = (yylsp[(2) - (3)]); + n->lhs = (yyvsp[-2].node); + n->rhs = (yyvsp[0].node); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *) n; - ;} + } +#line 26438 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 926: -#line 2353 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "->>", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); - ;} + case 946: /* a_expr: a_expr DOUBLE_ARROW a_expr */ +#line 2357 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "->>", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); + } +#line 26446 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 927: -#line 2357 "third_party/libpg_query/grammar/statements/select.y" - { - if (list_length((yyvsp[(1) - (3)].list)) != 2) + case 947: /* a_expr: row OVERLAPS row */ +#line 2361 "third_party/libpg_query/grammar/statements/select.y" + { + if (list_length((yyvsp[-2].list)) != 2) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("wrong number of parameters on left side of OVERLAPS expression"), - parser_errposition((yylsp[(1) - (3)])))); - if (list_length((yyvsp[(3) - (3)].list)) != 2) + parser_errposition((yylsp[-2])))); + if (list_length((yyvsp[0].list)) != 2) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("wrong number of parameters on right side of OVERLAPS expression"), - parser_errposition((yylsp[(3) - (3)])))); + parser_errposition((yylsp[0])))); (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("overlaps"), - list_concat((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)), - (yylsp[(2) - (3)])); - ;} + list_concat((yyvsp[-2].list), (yyvsp[0].list)), + (yylsp[-1])); + } +#line 26466 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 928: -#line 2373 "third_party/libpg_query/grammar/statements/select.y" - { + case 948: /* a_expr: a_expr IS TRUE_P */ +#line 2377 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (3)].node); + b->arg = (PGExpr *) (yyvsp[-2].node); b->booltesttype = PG_IS_TRUE; - b->location = (yylsp[(2) - (3)]); + b->location = (yylsp[-1]); (yyval.node) = (PGNode *)b; - ;} + } +#line 26478 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 929: -#line 2381 "third_party/libpg_query/grammar/statements/select.y" - { + case 949: /* a_expr: a_expr IS NOT TRUE_P */ +#line 2385 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (4)].node); + b->arg = (PGExpr *) (yyvsp[-3].node); b->booltesttype = IS_NOT_TRUE; - b->location = (yylsp[(2) - (4)]); + b->location = (yylsp[-2]); (yyval.node) = (PGNode *)b; - ;} + } +#line 26490 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 930: -#line 2389 "third_party/libpg_query/grammar/statements/select.y" - { + case 950: /* a_expr: a_expr IS FALSE_P */ +#line 2393 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (3)].node); + b->arg = (PGExpr *) (yyvsp[-2].node); b->booltesttype = IS_FALSE; - b->location = (yylsp[(2) - (3)]); + b->location = (yylsp[-1]); (yyval.node) = (PGNode *)b; - ;} + } +#line 26502 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 931: -#line 2397 "third_party/libpg_query/grammar/statements/select.y" - { + case 951: /* a_expr: a_expr IS NOT FALSE_P */ +#line 2401 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (4)].node); + b->arg = (PGExpr *) (yyvsp[-3].node); b->booltesttype = IS_NOT_FALSE; - b->location = (yylsp[(2) - (4)]); + b->location = (yylsp[-2]); (yyval.node) = (PGNode *)b; - ;} + } +#line 26514 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 932: -#line 2405 "third_party/libpg_query/grammar/statements/select.y" - { + case 952: /* a_expr: a_expr IS UNKNOWN */ +#line 2409 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (3)].node); + b->arg = (PGExpr *) (yyvsp[-2].node); b->booltesttype = IS_UNKNOWN; - b->location = (yylsp[(2) - (3)]); + b->location = (yylsp[-1]); (yyval.node) = (PGNode *)b; - ;} + } +#line 26526 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 933: -#line 2413 "third_party/libpg_query/grammar/statements/select.y" - { + case 953: /* a_expr: a_expr IS NOT UNKNOWN */ +#line 2417 "third_party/libpg_query/grammar/statements/select.y" + { PGBooleanTest *b = makeNode(PGBooleanTest); - b->arg = (PGExpr *) (yyvsp[(1) - (4)].node); + b->arg = (PGExpr *) (yyvsp[-3].node); b->booltesttype = IS_NOT_UNKNOWN; - b->location = (yylsp[(2) - (4)]); + b->location = (yylsp[-2]); (yyval.node) = (PGNode *)b; - ;} - break; - - case 934: -#line 2421 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_DISTINCT, "=", (yyvsp[(1) - (5)].node), (yyvsp[(5) - (5)].node), (yylsp[(2) - (5)])); - ;} + } +#line 26538 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 935: + case 954: /* a_expr: a_expr IS DISTINCT FROM a_expr */ #line 2425 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_DISTINCT, "=", (yyvsp[(1) - (6)].node), (yyvsp[(6) - (6)].node), (yylsp[(2) - (6)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_DISTINCT, "=", (yyvsp[-4].node), (yyvsp[0].node), (yylsp[-3])); + } +#line 26546 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 936: + case 955: /* a_expr: a_expr IS NOT DISTINCT FROM a_expr */ #line 2429 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "=", (yyvsp[(1) - (6)].node), (PGNode *) (yyvsp[(5) - (6)].list), (yylsp[(2) - (6)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_DISTINCT, "=", (yyvsp[-5].node), (yyvsp[0].node), (yylsp[-4])); + } +#line 26554 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 937: + case 956: /* a_expr: a_expr IS OF '(' type_list ')' */ #line 2433 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "<>", (yyvsp[(1) - (7)].node), (PGNode *) (yyvsp[(6) - (7)].list), (yylsp[(2) - (7)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "=", (yyvsp[-5].node), (PGNode *) (yyvsp[-1].list), (yylsp[-4])); + } +#line 26562 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 938: + case 957: /* a_expr: a_expr IS NOT OF '(' type_list ')' */ #line 2437 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "<>", (yyvsp[-6].node), (PGNode *) (yyvsp[-1].list), (yylsp[-5])); + } +#line 26570 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 958: /* a_expr: a_expr BETWEEN opt_asymmetric b_expr AND a_expr */ +#line 2441 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_BETWEEN, "BETWEEN", - (yyvsp[(1) - (6)].node), - (PGNode *) list_make2((yyvsp[(4) - (6)].node), (yyvsp[(6) - (6)].node)), - (yylsp[(2) - (6)])); - ;} + (yyvsp[-5].node), + (PGNode *) list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-4])); + } +#line 26582 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 939: -#line 2445 "third_party/libpg_query/grammar/statements/select.y" - { + case 959: /* a_expr: a_expr NOT_LA BETWEEN opt_asymmetric b_expr AND a_expr */ +#line 2449 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_BETWEEN, "NOT BETWEEN", - (yyvsp[(1) - (7)].node), - (PGNode *) list_make2((yyvsp[(5) - (7)].node), (yyvsp[(7) - (7)].node)), - (yylsp[(2) - (7)])); - ;} + (yyvsp[-6].node), + (PGNode *) list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-5])); + } +#line 26594 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 940: -#line 2453 "third_party/libpg_query/grammar/statements/select.y" - { + case 960: /* a_expr: a_expr BETWEEN SYMMETRIC b_expr AND a_expr */ +#line 2457 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_BETWEEN_SYM, "BETWEEN SYMMETRIC", - (yyvsp[(1) - (6)].node), - (PGNode *) list_make2((yyvsp[(4) - (6)].node), (yyvsp[(6) - (6)].node)), - (yylsp[(2) - (6)])); - ;} + (yyvsp[-5].node), + (PGNode *) list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-4])); + } +#line 26606 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 941: -#line 2461 "third_party/libpg_query/grammar/statements/select.y" - { + case 961: /* a_expr: a_expr NOT_LA BETWEEN SYMMETRIC b_expr AND a_expr */ +#line 2465 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_BETWEEN_SYM, "NOT BETWEEN SYMMETRIC", - (yyvsp[(1) - (7)].node), - (PGNode *) list_make2((yyvsp[(5) - (7)].node), (yyvsp[(7) - (7)].node)), - (yylsp[(2) - (7)])); - ;} + (yyvsp[-6].node), + (PGNode *) list_make2((yyvsp[-2].node), (yyvsp[0].node)), + (yylsp[-5])); + } +#line 26618 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 942: -#line 2469 "third_party/libpg_query/grammar/statements/select.y" - { + case 962: /* a_expr: a_expr IN_P in_expr */ +#line 2473 "third_party/libpg_query/grammar/statements/select.y" + { /* in_expr returns a PGSubLink or a list of a_exprs */ - if (IsA((yyvsp[(3) - (3)].node), PGSubLink)) + if (IsA((yyvsp[0].node), PGSubLink)) { /* generate foo = ANY (subquery) */ - PGSubLink *n = (PGSubLink *) (yyvsp[(3) - (3)].node); + PGSubLink *n = (PGSubLink *) (yyvsp[0].node); n->subLinkType = PG_ANY_SUBLINK; n->subLinkId = 0; - n->testexpr = (yyvsp[(1) - (3)].node); + n->testexpr = (yyvsp[-2].node); n->operName = NIL; /* show it's IN not = ANY */ - n->location = (yylsp[(2) - (3)]); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; } else { /* generate scalar IN expression */ - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_IN, "=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_IN, "=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } - ;} + } +#line 26642 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 943: -#line 2489 "third_party/libpg_query/grammar/statements/select.y" - { + case 963: /* a_expr: a_expr NOT_LA IN_P in_expr */ +#line 2493 "third_party/libpg_query/grammar/statements/select.y" + { /* in_expr returns a PGSubLink or a list of a_exprs */ - if (IsA((yyvsp[(4) - (4)].node), PGSubLink)) + if (IsA((yyvsp[0].node), PGSubLink)) { /* generate NOT (foo = ANY (subquery)) */ /* Make an = ANY node */ - PGSubLink *n = (PGSubLink *) (yyvsp[(4) - (4)].node); + PGSubLink *n = (PGSubLink *) (yyvsp[0].node); n->subLinkType = PG_ANY_SUBLINK; n->subLinkId = 0; - n->testexpr = (yyvsp[(1) - (4)].node); + n->testexpr = (yyvsp[-3].node); n->operName = NIL; /* show it's IN not = ANY */ - n->location = (yylsp[(2) - (4)]); + n->location = (yylsp[-2]); /* Stick a NOT on top; must have same parse location */ - (yyval.node) = makeNotExpr((PGNode *) n, (yylsp[(2) - (4)])); + (yyval.node) = makeNotExpr((PGNode *) n, (yylsp[-2])); } else { /* generate scalar NOT IN expression */ - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_IN, "<>", (yyvsp[(1) - (4)].node), (yyvsp[(4) - (4)].node), (yylsp[(2) - (4)])); + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_IN, "<>", (yyvsp[-3].node), (yyvsp[0].node), (yylsp[-2])); } - ;} + } +#line 26668 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 944: -#line 2511 "third_party/libpg_query/grammar/statements/select.y" - { + case 964: /* a_expr: a_expr subquery_Op sub_type select_with_parens */ +#line 2515 "third_party/libpg_query/grammar/statements/select.y" + { PGSubLink *n = makeNode(PGSubLink); - n->subLinkType = (yyvsp[(3) - (4)].subquerytype); + n->subLinkType = (yyvsp[-1].subquerytype); n->subLinkId = 0; - n->testexpr = (yyvsp[(1) - (4)].node); - n->operName = (yyvsp[(2) - (4)].list); - n->subselect = (yyvsp[(4) - (4)].node); - n->location = (yylsp[(2) - (4)]); + n->testexpr = (yyvsp[-3].node); + n->operName = (yyvsp[-2].list); + n->subselect = (yyvsp[0].node); + n->location = (yylsp[-2]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26683 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 945: -#line 2522 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(3) - (6)].subquerytype) == PG_ANY_SUBLINK) - (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP_ANY, (yyvsp[(2) - (6)].list), (yyvsp[(1) - (6)].node), (yyvsp[(5) - (6)].node), (yylsp[(2) - (6)])); + case 965: /* a_expr: a_expr subquery_Op sub_type '(' a_expr ')' */ +#line 2526 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[-3].subquerytype) == PG_ANY_SUBLINK) + (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP_ANY, (yyvsp[-4].list), (yyvsp[-5].node), (yyvsp[-1].node), (yylsp[-4])); else - (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP_ALL, (yyvsp[(2) - (6)].list), (yyvsp[(1) - (6)].node), (yyvsp[(5) - (6)].node), (yylsp[(2) - (6)])); - ;} + (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP_ALL, (yyvsp[-4].list), (yyvsp[-5].node), (yyvsp[-1].node), (yylsp[-4])); + } +#line 26694 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 946: -#line 2529 "third_party/libpg_query/grammar/statements/select.y" - { + case 966: /* a_expr: DEFAULT */ +#line 2533 "third_party/libpg_query/grammar/statements/select.y" + { /* * The SQL spec only allows DEFAULT in "contextually typed * expressions", but for us, it's easier to allow it in @@ -26048,288 +26705,329 @@ YYLTYPE yylloc; */ PGSetToDefault *n = makeNode(PGSetToDefault); /* parse analysis will fill in the rest */ - n->location = (yylsp[(1) - (1)]); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26712 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 947: -#line 2543 "third_party/libpg_query/grammar/statements/select.y" - { + case 967: /* a_expr: COLUMNS '(' a_expr ')' */ +#line 2547 "third_party/libpg_query/grammar/statements/select.y" + { PGAStar *star = makeNode(PGAStar); - star->expr = (yyvsp[(3) - (4)].node); + star->expr = (yyvsp[-1].node); star->columns = true; - star->location = (yylsp[(1) - (4)]); + star->location = (yylsp[-3]); (yyval.node) = (PGNode *) star; - ;} + } +#line 26724 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 948: -#line 2551 "third_party/libpg_query/grammar/statements/select.y" - { + case 968: /* a_expr: '*' opt_except_list opt_replace_list */ +#line 2555 "third_party/libpg_query/grammar/statements/select.y" + { PGAStar *star = makeNode(PGAStar); - star->except_list = (yyvsp[(2) - (3)].list); - star->replace_list = (yyvsp[(3) - (3)].list); - star->location = (yylsp[(1) - (3)]); + star->except_list = (yyvsp[-1].list); + star->replace_list = (yyvsp[0].list); + star->location = (yylsp[-2]); (yyval.node) = (PGNode *) star; - ;} + } +#line 26736 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 949: -#line 2559 "third_party/libpg_query/grammar/statements/select.y" - { + case 969: /* a_expr: ColId '.' '*' opt_except_list opt_replace_list */ +#line 2563 "third_party/libpg_query/grammar/statements/select.y" + { PGAStar *star = makeNode(PGAStar); - star->relation = (yyvsp[(1) - (5)].str); - star->except_list = (yyvsp[(4) - (5)].list); - star->replace_list = (yyvsp[(5) - (5)].list); - star->location = (yylsp[(1) - (5)]); + star->relation = (yyvsp[-4].str); + star->except_list = (yyvsp[-1].list); + star->replace_list = (yyvsp[0].list); + star->location = (yylsp[-4]); (yyval.node) = (PGNode *) star; - ;} - break; - - case 950: -#line 2579 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + } +#line 26749 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 951: -#line 2581 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeTypeCast((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].typnam), 0, (yylsp[(2) - (3)])); ;} - break; - - case 952: + case 970: /* b_expr: c_expr */ #line 2583 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 26755 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 953: + case 971: /* b_expr: b_expr TYPECAST Typename */ #line 2585 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = doNegate((yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + { (yyval.node) = makeTypeCast((yyvsp[-2].node), (yyvsp[0].typnam), 0, (yylsp[-1])); } +#line 26761 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 954: + case 972: /* b_expr: '+' b_expr */ #line 2587 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", NULL, (yyvsp[0].node), (yylsp[-1])); } +#line 26767 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 955: + case 973: /* b_expr: '-' b_expr */ #line 2589 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "-", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = doNegate((yyvsp[0].node), (yylsp[-1])); } +#line 26773 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 956: + case 974: /* b_expr: b_expr '+' b_expr */ #line 2591 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "*", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "+", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26779 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 957: + case 975: /* b_expr: b_expr '-' b_expr */ #line 2593 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "/", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "-", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26785 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 958: + case 976: /* b_expr: b_expr '*' b_expr */ #line 2595 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "//", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "*", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26791 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 959: + case 977: /* b_expr: b_expr '/' b_expr */ #line 2597 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "%", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "/", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26797 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 960: + case 978: /* b_expr: b_expr INTEGER_DIVISION b_expr */ #line 2599 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "^", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "//", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26803 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 961: + case 979: /* b_expr: b_expr '%' b_expr */ #line 2601 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "**", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "%", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26809 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 962: + case 980: /* b_expr: b_expr '^' b_expr */ #line 2603 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "^", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26815 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 963: + case 981: /* b_expr: b_expr POWER_OF b_expr */ #line 2605 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "**", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26821 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 964: + case 982: /* b_expr: b_expr '<' b_expr */ #line 2607 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26827 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 965: + case 983: /* b_expr: b_expr '>' b_expr */ #line 2609 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26833 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 966: + case 984: /* b_expr: b_expr '=' b_expr */ #line 2611 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">=", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26839 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 967: + case 985: /* b_expr: b_expr LESS_EQUALS b_expr */ #line 2613 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<>", (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26845 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 968: + case 986: /* b_expr: b_expr GREATER_EQUALS b_expr */ #line 2615 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(2) - (3)].list), (yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node), (yylsp[(2) - (3)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, ">=", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26851 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 969: + case 987: /* b_expr: b_expr NOT_EQUALS b_expr */ #line 2617 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(1) - (2)].list), NULL, (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); ;} + { (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OP, "<>", (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26857 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 970: + case 988: /* b_expr: b_expr qual_Op b_expr */ #line 2619 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[(2) - (2)].list), (yyvsp[(1) - (2)].node), NULL, (yylsp[(2) - (2)])); ;} + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[-1].list), (yyvsp[-2].node), (yyvsp[0].node), (yylsp[-1])); } +#line 26863 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 971: + case 989: /* b_expr: qual_Op b_expr */ #line 2621 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_DISTINCT, "=", (yyvsp[(1) - (5)].node), (yyvsp[(5) - (5)].node), (yylsp[(2) - (5)])); - ;} + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[-1].list), NULL, (yyvsp[0].node), (yylsp[-1])); } +#line 26869 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 990: /* b_expr: b_expr qual_Op */ +#line 2623 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeAExpr(PG_AEXPR_OP, (yyvsp[0].list), (yyvsp[-1].node), NULL, (yylsp[0])); } +#line 26875 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 972: + case 991: /* b_expr: b_expr IS DISTINCT FROM b_expr */ #line 2625 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_DISTINCT, "=", (yyvsp[(1) - (6)].node), (yyvsp[(6) - (6)].node), (yylsp[(2) - (6)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_DISTINCT, "=", (yyvsp[-4].node), (yyvsp[0].node), (yylsp[-3])); + } +#line 26883 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 973: + case 992: /* b_expr: b_expr IS NOT DISTINCT FROM b_expr */ #line 2629 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "=", (yyvsp[(1) - (6)].node), (PGNode *) (yyvsp[(5) - (6)].list), (yylsp[(2) - (6)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NOT_DISTINCT, "=", (yyvsp[-5].node), (yyvsp[0].node), (yylsp[-4])); + } +#line 26891 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 974: + case 993: /* b_expr: b_expr IS OF '(' type_list ')' */ #line 2633 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "<>", (yyvsp[(1) - (7)].node), (PGNode *) (yyvsp[(6) - (7)].list), (yylsp[(2) - (7)])); - ;} + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "=", (yyvsp[-5].node), (PGNode *) (yyvsp[-1].list), (yylsp[-4])); + } +#line 26899 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 976: -#line 2647 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall(SystemFuncName("row"), (yyvsp[(1) - (1)].list), (yylsp[(1) - (1)])); + case 994: /* b_expr: b_expr IS NOT OF '(' type_list ')' */ +#line 2637 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_OF, "<>", (yyvsp[-6].node), (PGNode *) (yyvsp[-1].list), (yylsp[-5])); + } +#line 26907 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 996: /* c_expr: row */ +#line 2651 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall(SystemFuncName("row"), (yyvsp[0].list), (yylsp[0])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26916 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 977: -#line 2652 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(2) - (2)].list)) + case 997: /* c_expr: indirection_expr opt_extended_indirection */ +#line 2656 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].list)) { PGAIndirection *n = makeNode(PGAIndirection); - n->arg = (PGNode *) (yyvsp[(1) - (2)].node); - n->indirection = check_indirection((yyvsp[(2) - (2)].list), yyscanner); + n->arg = (PGNode *) (yyvsp[-1].node); + n->indirection = check_indirection((yyvsp[0].list), yyscanner); (yyval.node) = (PGNode *) n; } else - (yyval.node) = (PGNode *) (yyvsp[(1) - (2)].node); - ;} + (yyval.node) = (PGNode *) (yyvsp[-1].node); + } +#line 26932 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 978: -#line 2665 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 998: /* d_expr: columnref */ +#line 2669 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 26938 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 979: -#line 2666 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 999: /* d_expr: AexprConst */ +#line 2670 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 26944 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 980: -#line 2668 "third_party/libpg_query/grammar/statements/select.y" - { + case 1000: /* d_expr: '#' ICONST */ +#line 2672 "third_party/libpg_query/grammar/statements/select.y" + { PGPositionalReference *n = makeNode(PGPositionalReference); - n->position = (yyvsp[(2) - (2)].ival); - n->location = (yylsp[(1) - (2)]); + n->position = (yyvsp[0].ival); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *) n; - ;} + } +#line 26955 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 981: -#line 2675 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeNamedParamRef((yyvsp[(2) - (2)].str), (yylsp[(1) - (2)])); - ;} + case 1001: /* d_expr: '$' ColLabel */ +#line 2679 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeNamedParamRef((yyvsp[0].str), (yylsp[-1])); + } +#line 26963 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 982: -#line 2678 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall(SystemFuncName("list_value"), (yyvsp[(2) - (3)].list), (yylsp[(2) - (3)])); + case 1002: /* d_expr: '[' opt_expr_list_opt_comma ']' */ +#line 2682 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall(SystemFuncName("list_value"), (yyvsp[-1].list), (yylsp[-1])); (yyval.node) = (PGNode *) n; - ;} + } +#line 26972 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 983: -#line 2682 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(1) - (1)].node); - ;} + case 1003: /* d_expr: list_comprehension */ +#line 2686 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (yyvsp[0].node); + } +#line 26980 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 984: -#line 2686 "third_party/libpg_query/grammar/statements/select.y" - { + case 1004: /* d_expr: ARRAY select_with_parens */ +#line 2690 "third_party/libpg_query/grammar/statements/select.y" + { PGSubLink *n = makeNode(PGSubLink); n->subLinkType = PG_ARRAY_SUBLINK; n->subLinkId = 0; n->testexpr = NULL; n->operName = NULL; - n->subselect = (yyvsp[(2) - (2)].node); - n->location = (yylsp[(2) - (2)]); + n->subselect = (yyvsp[0].node); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 26995 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 985: -#line 2696 "third_party/libpg_query/grammar/statements/select.y" - { + case 1005: /* d_expr: ARRAY '[' opt_expr_list_opt_comma ']' */ +#line 2700 "third_party/libpg_query/grammar/statements/select.y" + { PGList *func_name = list_make1(makeString("construct_array")); - PGFuncCall *n = makeFuncCall(func_name, (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); + PGFuncCall *n = makeFuncCall(func_name, (yyvsp[-1].list), (yylsp[-3])); (yyval.node) = (PGNode *) n; - ;} + } +#line 27005 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 986: -#line 2702 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 1006: /* d_expr: case_expr */ +#line 2706 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 27011 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 987: -#line 2704 "third_party/libpg_query/grammar/statements/select.y" - { + case 1007: /* d_expr: select_with_parens */ +#line 2708 "third_party/libpg_query/grammar/statements/select.y" + { PGSubLink *n = makeNode(PGSubLink); n->subLinkType = PG_EXPR_SUBLINK; n->subLinkId = 0; n->testexpr = NULL; n->operName = NIL; - n->subselect = (yyvsp[(1) - (1)].node); - n->location = (yylsp[(1) - (1)]); + n->subselect = (yyvsp[0].node); + n->location = (yylsp[0]); (yyval.node) = (PGNode *)n; - ;} + } +#line 27026 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 988: -#line 2715 "third_party/libpg_query/grammar/statements/select.y" - { + case 1008: /* d_expr: select_with_parens indirection */ +#line 2719 "third_party/libpg_query/grammar/statements/select.y" + { /* * Because the select_with_parens nonterminal is designed * to "eat" as many levels of parens as possible, the @@ -26346,172 +27044,188 @@ YYLTYPE yylloc; n->subLinkId = 0; n->testexpr = NULL; n->operName = NIL; - n->subselect = (yyvsp[(1) - (2)].node); - n->location = (yylsp[(1) - (2)]); + n->subselect = (yyvsp[-1].node); + n->location = (yylsp[-1]); a->arg = (PGNode *)n; - a->indirection = check_indirection((yyvsp[(2) - (2)].list), yyscanner); + a->indirection = check_indirection((yyvsp[0].list), yyscanner); (yyval.node) = (PGNode *)a; - ;} + } +#line 27054 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 989: -#line 2739 "third_party/libpg_query/grammar/statements/select.y" - { + case 1009: /* d_expr: EXISTS select_with_parens */ +#line 2743 "third_party/libpg_query/grammar/statements/select.y" + { PGSubLink *n = makeNode(PGSubLink); n->subLinkType = PG_EXISTS_SUBLINK; n->subLinkId = 0; n->testexpr = NULL; n->operName = NIL; - n->subselect = (yyvsp[(2) - (2)].node); - n->location = (yylsp[(1) - (2)]); + n->subselect = (yyvsp[0].node); + n->location = (yylsp[-1]); (yyval.node) = (PGNode *)n; - ;} + } +#line 27069 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 990: -#line 2750 "third_party/libpg_query/grammar/statements/select.y" - { + case 1010: /* d_expr: grouping_or_grouping_id '(' expr_list_opt_comma ')' */ +#line 2754 "third_party/libpg_query/grammar/statements/select.y" + { PGGroupingFunc *g = makeNode(PGGroupingFunc); - g->args = (yyvsp[(3) - (4)].list); - g->location = (yylsp[(1) - (4)]); + g->args = (yyvsp[-1].list); + g->location = (yylsp[-3]); (yyval.node) = (PGNode *)g; - ;} + } +#line 27080 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 991: -#line 2761 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeParamRef(0, (yylsp[(1) - (1)])); - ;} + case 1011: /* indirection_expr: '?' */ +#line 2765 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeParamRef(0, (yylsp[0])); + } +#line 27088 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 992: -#line 2765 "third_party/libpg_query/grammar/statements/select.y" - { + case 1012: /* indirection_expr: PARAM */ +#line 2769 "third_party/libpg_query/grammar/statements/select.y" + { PGParamRef *p = makeNode(PGParamRef); - p->number = (yyvsp[(1) - (1)].ival); - p->location = (yylsp[(1) - (1)]); + p->number = (yyvsp[0].ival); + p->location = (yylsp[0]); (yyval.node) = (PGNode *) p; - ;} - break; - - case 993: -#line 2772 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(2) - (3)].node); - ;} + } +#line 27099 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 994: + case 1013: /* indirection_expr: '(' a_expr ')' */ #line 2776 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(1) - (1)].node); - ;} + { + (yyval.node) = (yyvsp[-1].node); + } +#line 27107 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 995: + case 1014: /* indirection_expr: struct_expr */ #line 2780 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.node) = (yyvsp[0].node); + } +#line 27115 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1015: /* indirection_expr: MAP '{' opt_map_arguments_opt_comma '}' */ +#line 2784 "third_party/libpg_query/grammar/statements/select.y" + { PGList *key_list = NULL; PGList *value_list = NULL; PGListCell *lc; - PGList *entry_list = (yyvsp[(3) - (4)].list); + PGList *entry_list = (yyvsp[-1].list); foreach(lc, entry_list) { PGList *l = (PGList *) lc->data.ptr_value; key_list = lappend(key_list, (PGNode *) l->head->data.ptr_value); value_list = lappend(value_list, (PGNode *) l->tail->data.ptr_value); } - PGNode *keys = (PGNode *) makeFuncCall(SystemFuncName("list_value"), key_list, (yylsp[(3) - (4)])); - PGNode *values = (PGNode *) makeFuncCall(SystemFuncName("list_value"), value_list, (yylsp[(3) - (4)])); - PGFuncCall *f = makeFuncCall(SystemFuncName("map"), list_make2(keys, values), (yylsp[(3) - (4)])); + PGNode *keys = (PGNode *) makeFuncCall(SystemFuncName("list_value"), key_list, (yylsp[-1])); + PGNode *values = (PGNode *) makeFuncCall(SystemFuncName("list_value"), value_list, (yylsp[-1])); + PGFuncCall *f = makeFuncCall(SystemFuncName("map"), list_make2(keys, values), (yylsp[-1])); (yyval.node) = (PGNode *) f; - ;} + } +#line 27136 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 996: -#line 2797 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(1) - (1)].node); - ;} + case 1016: /* indirection_expr: func_expr */ +#line 2801 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (yyvsp[0].node); + } +#line 27144 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 997: -#line 2805 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *f = makeFuncCall(SystemFuncName("struct_pack"), (yyvsp[(2) - (3)].list), (yylsp[(2) - (3)])); + case 1017: /* struct_expr: '{' dict_arguments_opt_comma '}' */ +#line 2809 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *f = makeFuncCall(SystemFuncName("struct_pack"), (yyvsp[-1].list), (yylsp[-1])); (yyval.node) = (PGNode *) f; - ;} - break; - - case 998: -#line 2814 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeFuncCall((yyvsp[(1) - (3)].list), NIL, (yylsp[(1) - (3)])); - ;} + } +#line 27153 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 999: + case 1018: /* func_application: func_name '(' ')' */ #line 2818 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall((yyvsp[(1) - (6)].list), (yyvsp[(3) - (6)].list), (yylsp[(1) - (6)])); - n->agg_order = (yyvsp[(4) - (6)].list); - n->agg_ignore_nulls = (yyvsp[(5) - (6)].boolean); + { + (yyval.node) = (PGNode *) makeFuncCall((yyvsp[-2].list), NIL, (yylsp[-2])); + } +#line 27161 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1019: /* func_application: func_name '(' func_arg_list opt_sort_clause opt_ignore_nulls ')' */ +#line 2822 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall((yyvsp[-5].list), (yyvsp[-3].list), (yylsp[-5])); + n->agg_order = (yyvsp[-2].list); + n->agg_ignore_nulls = (yyvsp[-1].boolean); (yyval.node) = (PGNode *)n; - ;} + } +#line 27172 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1000: -#line 2825 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall((yyvsp[(1) - (7)].list), list_make1((yyvsp[(4) - (7)].node)), (yylsp[(1) - (7)])); + case 1020: /* func_application: func_name '(' VARIADIC func_arg_expr opt_sort_clause opt_ignore_nulls ')' */ +#line 2829 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall((yyvsp[-6].list), list_make1((yyvsp[-3].node)), (yylsp[-6])); n->func_variadic = true; - n->agg_order = (yyvsp[(5) - (7)].list); - n->agg_ignore_nulls = (yyvsp[(6) - (7)].boolean); + n->agg_order = (yyvsp[-2].list); + n->agg_ignore_nulls = (yyvsp[-1].boolean); (yyval.node) = (PGNode *)n; - ;} + } +#line 27184 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1001: -#line 2833 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall((yyvsp[(1) - (9)].list), lappend((yyvsp[(3) - (9)].list), (yyvsp[(6) - (9)].node)), (yylsp[(1) - (9)])); + case 1021: /* func_application: func_name '(' func_arg_list ',' VARIADIC func_arg_expr opt_sort_clause opt_ignore_nulls ')' */ +#line 2837 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall((yyvsp[-8].list), lappend((yyvsp[-6].list), (yyvsp[-3].node)), (yylsp[-8])); n->func_variadic = true; - n->agg_order = (yyvsp[(7) - (9)].list); - n->agg_ignore_nulls = (yyvsp[(8) - (9)].boolean); + n->agg_order = (yyvsp[-2].list); + n->agg_ignore_nulls = (yyvsp[-1].boolean); (yyval.node) = (PGNode *)n; - ;} + } +#line 27196 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1002: -#line 2841 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall((yyvsp[(1) - (7)].list), (yyvsp[(4) - (7)].list), (yylsp[(1) - (7)])); - n->agg_order = (yyvsp[(5) - (7)].list); - n->agg_ignore_nulls = (yyvsp[(6) - (7)].boolean); + case 1022: /* func_application: func_name '(' ALL func_arg_list opt_sort_clause opt_ignore_nulls ')' */ +#line 2845 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall((yyvsp[-6].list), (yyvsp[-3].list), (yylsp[-6])); + n->agg_order = (yyvsp[-2].list); + n->agg_ignore_nulls = (yyvsp[-1].boolean); /* Ideally we'd mark the PGFuncCall node to indicate * "must be an aggregate", but there's no provision * for that in PGFuncCall at the moment. */ (yyval.node) = (PGNode *)n; - ;} + } +#line 27211 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1003: -#line 2852 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = makeFuncCall((yyvsp[(1) - (7)].list), (yyvsp[(4) - (7)].list), (yylsp[(1) - (7)])); - n->agg_order = (yyvsp[(5) - (7)].list); - n->agg_ignore_nulls = (yyvsp[(6) - (7)].boolean); + case 1023: /* func_application: func_name '(' DISTINCT func_arg_list opt_sort_clause opt_ignore_nulls ')' */ +#line 2856 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = makeFuncCall((yyvsp[-6].list), (yyvsp[-3].list), (yylsp[-6])); + n->agg_order = (yyvsp[-2].list); + n->agg_ignore_nulls = (yyvsp[-1].boolean); n->agg_distinct = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 27223 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1004: -#line 2872 "third_party/libpg_query/grammar/statements/select.y" - { - PGFuncCall *n = (PGFuncCall *) (yyvsp[(1) - (5)].node); + case 1024: /* func_expr: func_application within_group_clause filter_clause export_clause over_clause */ +#line 2876 "third_party/libpg_query/grammar/statements/select.y" + { + PGFuncCall *n = (PGFuncCall *) (yyvsp[-4].node); /* * The order clause for WITHIN GROUP and the one for * plain-aggregate ORDER BY share a field, so we have to @@ -26520,107 +27234,118 @@ YYLTYPE yylloc; * location. Other consistency checks are deferred to * parse analysis. */ - if ((yyvsp[(2) - (5)].list) != NIL) + if ((yyvsp[-3].list) != NIL) { if (n->agg_order != NIL) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("cannot use multiple ORDER BY clauses with WITHIN GROUP"), - parser_errposition((yylsp[(2) - (5)])))); + parser_errposition((yylsp[-3])))); if (n->agg_distinct) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("cannot use DISTINCT with WITHIN GROUP"), - parser_errposition((yylsp[(2) - (5)])))); + parser_errposition((yylsp[-3])))); if (n->func_variadic) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("cannot use VARIADIC with WITHIN GROUP"), - parser_errposition((yylsp[(2) - (5)])))); - n->agg_order = (yyvsp[(2) - (5)].list); + parser_errposition((yylsp[-3])))); + n->agg_order = (yyvsp[-3].list); n->agg_within_group = true; } - n->agg_filter = (yyvsp[(3) - (5)].node); - n->export_state = (yyvsp[(4) - (5)].boolean); - n->over = (yyvsp[(5) - (5)].windef); + n->agg_filter = (yyvsp[-2].node); + n->export_state = (yyvsp[-1].boolean); + n->over = (yyvsp[0].windef); (yyval.node) = (PGNode *) n; - ;} + } +#line 27263 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1005: -#line 2908 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 1025: /* func_expr: func_expr_common_subexpr */ +#line 2912 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 27269 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1006: -#line 2918 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 1026: /* func_expr_windowless: func_application */ +#line 2922 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 27275 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1007: -#line 2919 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 1027: /* func_expr_windowless: func_expr_common_subexpr */ +#line 2923 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 27281 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1008: -#line 2927 "third_party/libpg_query/grammar/statements/select.y" - { + case 1028: /* func_expr_common_subexpr: COLLATION FOR '(' a_expr ')' */ +#line 2931 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("pg_collation_for"), - list_make1((yyvsp[(4) - (5)].node)), - (yylsp[(1) - (5)])); - ;} + list_make1((yyvsp[-1].node)), + (yylsp[-4])); + } +#line 27291 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1009: -#line 2933 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeTypeCast((yyvsp[(3) - (6)].node), (yyvsp[(5) - (6)].typnam), 0, (yylsp[(1) - (6)])); ;} + case 1029: /* func_expr_common_subexpr: CAST '(' a_expr AS Typename ')' */ +#line 2937 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeTypeCast((yyvsp[-3].node), (yyvsp[-1].typnam), 0, (yylsp[-5])); } +#line 27297 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1010: -#line 2935 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = makeTypeCast((yyvsp[(3) - (6)].node), (yyvsp[(5) - (6)].typnam), 1, (yylsp[(1) - (6)])); ;} + case 1030: /* func_expr_common_subexpr: TRY_CAST '(' a_expr AS Typename ')' */ +#line 2939 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = makeTypeCast((yyvsp[-3].node), (yyvsp[-1].typnam), 1, (yylsp[-5])); } +#line 27303 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1011: -#line 2937 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("date_part"), (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + case 1031: /* func_expr_common_subexpr: EXTRACT '(' extract_list ')' */ +#line 2941 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("date_part"), (yyvsp[-1].list), (yylsp[-3])); + } +#line 27311 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1012: -#line 2941 "third_party/libpg_query/grammar/statements/select.y" - { + case 1032: /* func_expr_common_subexpr: OVERLAY '(' overlay_list ')' */ +#line 2945 "third_party/libpg_query/grammar/statements/select.y" + { /* overlay(A PLACING B FROM C FOR D) is converted to * overlay(A, B, C, D) * overlay(A PLACING B FROM C) is converted to * overlay(A, B, C) */ - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("overlay"), (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("overlay"), (yyvsp[-1].list), (yylsp[-3])); + } +#line 27324 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1013: -#line 2950 "third_party/libpg_query/grammar/statements/select.y" - { + case 1033: /* func_expr_common_subexpr: POSITION '(' position_list ')' */ +#line 2954 "third_party/libpg_query/grammar/statements/select.y" + { /* position(A in B) is converted to position(B, A) */ - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("position"), (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("position"), (yyvsp[-1].list), (yylsp[-3])); + } +#line 27333 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1014: -#line 2955 "third_party/libpg_query/grammar/statements/select.y" - { + case 1034: /* func_expr_common_subexpr: SUBSTRING '(' substr_list ')' */ +#line 2959 "third_party/libpg_query/grammar/statements/select.y" + { /* substring(A from B for C) is converted to * substring(A, B, C) - thomas 2000-11-28 */ - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("substring"), (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("substring"), (yyvsp[-1].list), (yylsp[-3])); + } +#line 27344 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1015: -#line 2962 "third_party/libpg_query/grammar/statements/select.y" - { + case 1035: /* func_expr_common_subexpr: TREAT '(' a_expr AS Typename ')' */ +#line 2966 "third_party/libpg_query/grammar/statements/select.y" + { /* TREAT(expr AS target) converts expr of a particular type to target, * which is defined to be a subtype of the original expression. * In SQL99, this is intended for use with structured UDTs, @@ -26630,288 +27355,322 @@ YYLTYPE yylloc; * Convert SystemTypeName() to SystemFuncName() even though * at the moment they result in the same thing. */ - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName(((PGValue *)llast((yyvsp[(5) - (6)].typnam)->names))->val.str), - list_make1((yyvsp[(3) - (6)].node)), - (yylsp[(1) - (6)])); - ;} + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName(((PGValue *)llast((yyvsp[-1].typnam)->names))->val.str), + list_make1((yyvsp[-3].node)), + (yylsp[-5])); + } +#line 27363 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1016: -#line 2977 "third_party/libpg_query/grammar/statements/select.y" - { + case 1036: /* func_expr_common_subexpr: TRIM '(' BOTH trim_list ')' */ +#line 2981 "third_party/libpg_query/grammar/statements/select.y" + { /* various trim expressions are defined in SQL * - thomas 1997-07-19 */ - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("trim"), (yyvsp[(4) - (5)].list), (yylsp[(1) - (5)])); - ;} - break; - - case 1017: -#line 2984 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("ltrim"), (yyvsp[(4) - (5)].list), (yylsp[(1) - (5)])); - ;} + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("trim"), (yyvsp[-1].list), (yylsp[-4])); + } +#line 27374 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1018: + case 1037: /* func_expr_common_subexpr: TRIM '(' LEADING trim_list ')' */ #line 2988 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("rtrim"), (yyvsp[(4) - (5)].list), (yylsp[(1) - (5)])); - ;} + { + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("ltrim"), (yyvsp[-1].list), (yylsp[-4])); + } +#line 27382 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1019: + case 1038: /* func_expr_common_subexpr: TRIM '(' TRAILING trim_list ')' */ #line 2992 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("trim"), (yyvsp[(3) - (4)].list), (yylsp[(1) - (4)])); - ;} + { + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("rtrim"), (yyvsp[-1].list), (yylsp[-4])); + } +#line 27390 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1020: + case 1039: /* func_expr_common_subexpr: TRIM '(' trim_list ')' */ #line 2996 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NULLIF, "=", (yyvsp[(3) - (6)].node), (yyvsp[(5) - (6)].node), (yylsp[(1) - (6)])); - ;} + { + (yyval.node) = (PGNode *) makeFuncCall(SystemFuncName("trim"), (yyvsp[-1].list), (yylsp[-3])); + } +#line 27398 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1021: + case 1040: /* func_expr_common_subexpr: NULLIF '(' a_expr ',' a_expr ')' */ #line 3000 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.node) = (PGNode *) makeSimpleAExpr(PG_AEXPR_NULLIF, "=", (yyvsp[-3].node), (yyvsp[-1].node), (yylsp[-5])); + } +#line 27406 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1041: /* func_expr_common_subexpr: COALESCE '(' expr_list_opt_comma ')' */ +#line 3004 "third_party/libpg_query/grammar/statements/select.y" + { PGCoalesceExpr *c = makeNode(PGCoalesceExpr); - c->args = (yyvsp[(3) - (4)].list); - c->location = (yylsp[(1) - (4)]); + c->args = (yyvsp[-1].list); + c->location = (yylsp[-3]); (yyval.node) = (PGNode *)c; - ;} + } +#line 27417 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1022: -#line 3010 "third_party/libpg_query/grammar/statements/select.y" - { + case 1042: /* list_comprehension: '[' a_expr FOR ColId IN_P a_expr ']' */ +#line 3014 "third_party/libpg_query/grammar/statements/select.y" + { PGLambdaFunction *lambda = makeNode(PGLambdaFunction); - lambda->lhs = makeColumnRef((yyvsp[(4) - (7)].str), NIL, (yylsp[(4) - (7)]), yyscanner); - lambda->rhs = (yyvsp[(2) - (7)].node); - lambda->location = (yylsp[(1) - (7)]); - PGFuncCall *n = makeFuncCall(SystemFuncName("list_apply"), list_make2((yyvsp[(6) - (7)].node), lambda), (yylsp[(1) - (7)])); + lambda->lhs = makeColumnRef((yyvsp[-3].str), NIL, (yylsp[-3]), yyscanner); + lambda->rhs = (yyvsp[-5].node); + lambda->location = (yylsp[-6]); + PGFuncCall *n = makeFuncCall(SystemFuncName("list_apply"), list_make2((yyvsp[-1].node), lambda), (yylsp[-6])); (yyval.node) = (PGNode *) n; - ;} + } +#line 27430 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1023: -#line 3019 "third_party/libpg_query/grammar/statements/select.y" - { + case 1043: /* list_comprehension: '[' a_expr FOR ColId IN_P c_expr IF_P a_expr ']' */ +#line 3023 "third_party/libpg_query/grammar/statements/select.y" + { PGLambdaFunction *lambda = makeNode(PGLambdaFunction); - lambda->lhs = makeColumnRef((yyvsp[(4) - (9)].str), NIL, (yylsp[(4) - (9)]), yyscanner); - lambda->rhs = (yyvsp[(2) - (9)].node); - lambda->location = (yylsp[(1) - (9)]); + lambda->lhs = makeColumnRef((yyvsp[-5].str), NIL, (yylsp[-5]), yyscanner); + lambda->rhs = (yyvsp[-7].node); + lambda->location = (yylsp[-8]); PGLambdaFunction *lambda_filter = makeNode(PGLambdaFunction); - lambda_filter->lhs = makeColumnRef((yyvsp[(4) - (9)].str), NIL, (yylsp[(4) - (9)]), yyscanner); - lambda_filter->rhs = (yyvsp[(8) - (9)].node); - lambda_filter->location = (yylsp[(8) - (9)]); - PGFuncCall *filter = makeFuncCall(SystemFuncName("list_filter"), list_make2((yyvsp[(6) - (9)].node), lambda_filter), (yylsp[(1) - (9)])); - PGFuncCall *n = makeFuncCall(SystemFuncName("list_apply"), list_make2(filter, lambda), (yylsp[(1) - (9)])); + lambda_filter->lhs = makeColumnRef((yyvsp[-5].str), NIL, (yylsp[-5]), yyscanner); + lambda_filter->rhs = (yyvsp[-1].node); + lambda_filter->location = (yylsp[-1]); + PGFuncCall *filter = makeFuncCall(SystemFuncName("list_filter"), list_make2((yyvsp[-3].node), lambda_filter), (yylsp[-8])); + PGFuncCall *n = makeFuncCall(SystemFuncName("list_apply"), list_make2(filter, lambda), (yylsp[-8])); (yyval.node) = (PGNode *) n; - ;} - break; - - case 1024: -#line 3040 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(4) - (5)].list); ;} + } +#line 27449 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1025: -#line 3041 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1044: /* within_group_clause: WITHIN GROUP_P '(' sort_clause ')' */ +#line 3044 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 27455 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1026: + case 1045: /* within_group_clause: %empty */ #line 3045 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(4) - (5)].node); ;} + { (yyval.list) = NIL; } +#line 27461 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1027: -#line 3046 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(3) - (4)].node); ;} + case 1046: /* filter_clause: FILTER '(' WHERE a_expr ')' */ +#line 3049 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[-1].node); } +#line 27467 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1028: -#line 3047 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 1047: /* filter_clause: FILTER '(' a_expr ')' */ +#line 3050 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[-1].node); } +#line 27473 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1029: + case 1048: /* filter_clause: %empty */ #line 3051 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = true; ;} + { (yyval.node) = NULL; } +#line 27479 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1030: -#line 3052 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.boolean) = false; ;} + case 1049: /* export_clause: EXPORT_STATE */ +#line 3055 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = true; } +#line 27485 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1031: -#line 3059 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + case 1050: /* export_clause: %empty */ +#line 3056 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.boolean) = false; } +#line 27491 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1032: -#line 3060 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1051: /* window_clause: WINDOW window_definition_list */ +#line 3063 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 27497 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1033: + case 1052: /* window_clause: %empty */ #line 3064 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].windef)); ;} + { (yyval.list) = NIL; } +#line 27503 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1034: -#line 3066 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].windef)); ;} + case 1053: /* window_definition_list: window_definition */ +#line 3068 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].windef)); } +#line 27509 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1035: -#line 3071 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n = (yyvsp[(3) - (3)].windef); - n->name = (yyvsp[(1) - (3)].str); + case 1054: /* window_definition_list: window_definition_list ',' window_definition */ +#line 3070 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].windef)); } +#line 27515 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1055: /* window_definition: ColId AS window_specification */ +#line 3075 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[0].windef); + n->name = (yyvsp[-2].str); (yyval.windef) = n; - ;} + } +#line 27525 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1036: -#line 3079 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.windef) = (yyvsp[(2) - (2)].windef); ;} + case 1056: /* over_clause: OVER window_specification */ +#line 3083 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.windef) = (yyvsp[0].windef); } +#line 27531 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1037: -#line 3081 "third_party/libpg_query/grammar/statements/select.y" - { + case 1057: /* over_clause: OVER ColId */ +#line 3085 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); n->refname = NULL; n->partitionClause = NIL; n->orderClause = NIL; n->frameOptions = FRAMEOPTION_DEFAULTS; n->startOffset = NULL; n->endOffset = NULL; - n->location = (yylsp[(2) - (2)]); + n->location = (yylsp[0]); (yyval.windef) = n; - ;} + } +#line 27548 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1038: -#line 3094 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.windef) = NULL; ;} + case 1058: /* over_clause: %empty */ +#line 3098 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.windef) = NULL; } +#line 27554 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1039: -#line 3099 "third_party/libpg_query/grammar/statements/select.y" - { + case 1059: /* window_specification: '(' opt_existing_window_name opt_partition_clause opt_sort_clause opt_frame_clause ')' */ +#line 3103 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->name = NULL; - n->refname = (yyvsp[(2) - (6)].str); - n->partitionClause = (yyvsp[(3) - (6)].list); - n->orderClause = (yyvsp[(4) - (6)].list); + n->refname = (yyvsp[-4].str); + n->partitionClause = (yyvsp[-3].list); + n->orderClause = (yyvsp[-2].list); /* copy relevant fields of opt_frame_clause */ - n->frameOptions = (yyvsp[(5) - (6)].windef)->frameOptions; - n->startOffset = (yyvsp[(5) - (6)].windef)->startOffset; - n->endOffset = (yyvsp[(5) - (6)].windef)->endOffset; - n->location = (yylsp[(1) - (6)]); + n->frameOptions = (yyvsp[-1].windef)->frameOptions; + n->startOffset = (yyvsp[-1].windef)->startOffset; + n->endOffset = (yyvsp[-1].windef)->endOffset; + n->location = (yylsp[-5]); (yyval.windef) = n; - ;} + } +#line 27572 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1040: -#line 3124 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 1060: /* opt_existing_window_name: ColId */ +#line 3128 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 27578 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1041: -#line 3125 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = NULL; ;} + case 1061: /* opt_existing_window_name: %empty */ +#line 3129 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = NULL; } +#line 27584 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1042: -#line 3128 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (3)].list); ;} + case 1062: /* opt_partition_clause: PARTITION BY expr_list */ +#line 3132 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 27590 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1043: -#line 3129 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1063: /* opt_partition_clause: %empty */ +#line 3133 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 27596 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1044: -#line 3138 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n = (yyvsp[(2) - (3)].windef); + case 1064: /* opt_frame_clause: RANGE frame_extent opt_window_exclusion_clause */ +#line 3142 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[-1].windef); n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_RANGE; - n->frameOptions |= (yyvsp[(3) - (3)].ival); + n->frameOptions |= (yyvsp[0].ival); (yyval.windef) = n; - ;} + } +#line 27608 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1045: -#line 3146 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n = (yyvsp[(2) - (3)].windef); + case 1065: /* opt_frame_clause: ROWS frame_extent opt_window_exclusion_clause */ +#line 3150 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[-1].windef); n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_ROWS; - n->frameOptions |= (yyvsp[(3) - (3)].ival); + n->frameOptions |= (yyvsp[0].ival); (yyval.windef) = n; - ;} + } +#line 27620 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1046: -#line 3154 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n = (yyvsp[(2) - (3)].windef); + case 1066: /* opt_frame_clause: GROUPS frame_extent opt_window_exclusion_clause */ +#line 3158 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[-1].windef); n->frameOptions |= FRAMEOPTION_NONDEFAULT | FRAMEOPTION_GROUPS; - n->frameOptions |= (yyvsp[(3) - (3)].ival); + n->frameOptions |= (yyvsp[0].ival); (yyval.windef) = n; - ;} + } +#line 27632 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1047: -#line 3162 "third_party/libpg_query/grammar/statements/select.y" - { + case 1067: /* opt_frame_clause: %empty */ +#line 3166 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_DEFAULTS; n->startOffset = NULL; n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27645 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1048: -#line 3173 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n = (yyvsp[(1) - (1)].windef); + case 1068: /* frame_extent: frame_bound */ +#line 3177 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n = (yyvsp[0].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)])))); + parser_errposition((yylsp[0])))); 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"), - parser_errposition((yylsp[(1) - (1)])))); + parser_errposition((yylsp[0])))); n->frameOptions |= FRAMEOPTION_END_CURRENT_ROW; (yyval.windef) = n; - ;} + } +#line 27667 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1049: -#line 3191 "third_party/libpg_query/grammar/statements/select.y" - { - PGWindowDef *n1 = (yyvsp[(2) - (4)].windef); - PGWindowDef *n2 = (yyvsp[(4) - (4)].windef); + case 1069: /* frame_extent: BETWEEN frame_bound AND frame_bound */ +#line 3195 "third_party/libpg_query/grammar/statements/select.y" + { + PGWindowDef *n1 = (yyvsp[-2].windef); + PGWindowDef *n2 = (yyvsp[0].windef); /* form merged options */ int frameOptions = n1->frameOptions; @@ -26923,631 +27682,732 @@ YYLTYPE yylloc; ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame start cannot be UNBOUNDED FOLLOWING"), - parser_errposition((yylsp[(2) - (4)])))); + parser_errposition((yylsp[-2])))); if (frameOptions & FRAMEOPTION_END_UNBOUNDED_PRECEDING) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame end cannot be UNBOUNDED PRECEDING"), - parser_errposition((yylsp[(4) - (4)])))); + parser_errposition((yylsp[0])))); if ((frameOptions & FRAMEOPTION_START_CURRENT_ROW) && (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)])))); + parser_errposition((yylsp[0])))); if ((frameOptions & FRAMEOPTION_START_OFFSET_FOLLOWING) && (frameOptions & (FRAMEOPTION_END_OFFSET_PRECEDING | FRAMEOPTION_END_CURRENT_ROW))) ereport(ERROR, (errcode(PG_ERRCODE_WINDOWING_ERROR), errmsg("frame starting from following row cannot have preceding rows"), - parser_errposition((yylsp[(4) - (4)])))); + parser_errposition((yylsp[0])))); n1->frameOptions = frameOptions; n1->endOffset = n2->startOffset; (yyval.windef) = n1; - ;} + } +#line 27709 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1050: -#line 3237 "third_party/libpg_query/grammar/statements/select.y" - { + case 1070: /* frame_bound: UNBOUNDED PRECEDING */ +#line 3241 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_START_UNBOUNDED_PRECEDING; n->startOffset = NULL; n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27722 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1051: -#line 3246 "third_party/libpg_query/grammar/statements/select.y" - { + case 1071: /* frame_bound: UNBOUNDED FOLLOWING */ +#line 3250 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_START_UNBOUNDED_FOLLOWING; n->startOffset = NULL; n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27735 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1052: -#line 3255 "third_party/libpg_query/grammar/statements/select.y" - { + case 1072: /* frame_bound: CURRENT_P ROW */ +#line 3259 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_START_CURRENT_ROW; n->startOffset = NULL; n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27748 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1053: -#line 3264 "third_party/libpg_query/grammar/statements/select.y" - { + case 1073: /* frame_bound: a_expr PRECEDING */ +#line 3268 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_START_OFFSET_PRECEDING; - n->startOffset = (yyvsp[(1) - (2)].node); + n->startOffset = (yyvsp[-1].node); n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27761 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1054: -#line 3273 "third_party/libpg_query/grammar/statements/select.y" - { + case 1074: /* frame_bound: a_expr FOLLOWING */ +#line 3277 "third_party/libpg_query/grammar/statements/select.y" + { PGWindowDef *n = makeNode(PGWindowDef); n->frameOptions = FRAMEOPTION_START_OFFSET_FOLLOWING; - n->startOffset = (yyvsp[(1) - (2)].node); + n->startOffset = (yyvsp[-1].node); n->endOffset = NULL; (yyval.windef) = n; - ;} + } +#line 27774 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1055: -#line 3284 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = FRAMEOPTION_EXCLUDE_CURRENT_ROW; ;} + case 1075: /* opt_window_exclusion_clause: EXCLUDE CURRENT_P ROW */ +#line 3288 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_CURRENT_ROW; } +#line 27780 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1056: -#line 3285 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = FRAMEOPTION_EXCLUDE_GROUP; ;} + case 1076: /* opt_window_exclusion_clause: EXCLUDE GROUP_P */ +#line 3289 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_GROUP; } +#line 27786 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1057: -#line 3286 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = FRAMEOPTION_EXCLUDE_TIES; ;} + case 1077: /* opt_window_exclusion_clause: EXCLUDE TIES */ +#line 3290 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = FRAMEOPTION_EXCLUDE_TIES; } +#line 27792 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1058: -#line 3287 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} + case 1078: /* opt_window_exclusion_clause: EXCLUDE NO OTHERS */ +#line 3291 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; } +#line 27798 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1059: -#line 3288 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = 0; ;} + case 1079: /* opt_window_exclusion_clause: %empty */ +#line 3292 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = 0; } +#line 27804 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1060: -#line 3302 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + case 1080: /* qualified_row: ROW '(' expr_list_opt_comma ')' */ +#line 3306 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 27810 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1061: -#line 3303 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1081: /* qualified_row: ROW '(' ')' */ +#line 3307 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 27816 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1062: -#line 3306 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list);;} + case 1082: /* row: qualified_row */ +#line 3310 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list);} +#line 27822 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1063: -#line 3307 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(2) - (5)].list), (yyvsp[(4) - (5)].node)); ;} + case 1083: /* row: '(' expr_list ',' a_expr ')' */ +#line 3311 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-3].list), (yyvsp[-1].node)); } +#line 27828 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1064: -#line 3311 "third_party/libpg_query/grammar/statements/select.y" - { + case 1084: /* dict_arg: ColIdOrString ':' a_expr */ +#line 3315 "third_party/libpg_query/grammar/statements/select.y" + { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); - na->name = (yyvsp[(1) - (3)].str); - na->arg = (PGExpr *) (yyvsp[(3) - (3)].node); + na->name = (yyvsp[-2].str); + na->arg = (PGExpr *) (yyvsp[0].node); na->argnumber = -1; - na->location = (yylsp[(1) - (3)]); + na->location = (yylsp[-2]); (yyval.node) = (PGNode *) na; - ;} - break; - - case 1065: -#line 3321 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + } +#line 27841 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1066: -#line 3322 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + case 1085: /* dict_arguments: dict_arg */ +#line 3325 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 27847 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1067: + case 1086: /* dict_arguments: dict_arguments ',' dict_arg */ #line 3326 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 27853 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1068: -#line 3327 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 1087: /* dict_arguments_opt_comma: dict_arguments */ +#line 3330 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 27859 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1069: -#line 3332 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make2((yyvsp[(1) - (3)].node), (yyvsp[(3) - (3)].node)); - ;} - break; - - case 1070: -#line 3338 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} + case 1088: /* dict_arguments_opt_comma: dict_arguments ',' */ +#line 3331 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 27865 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1071: -#line 3339 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} + case 1089: /* map_arg: a_expr ':' a_expr */ +#line 3336 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make2((yyvsp[-2].node), (yyvsp[0].node)); + } +#line 27873 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1072: -#line 3344 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1090: /* map_arguments: map_arg */ +#line 3342 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].list)); } +#line 27879 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1073: -#line 3345 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 1091: /* map_arguments: map_arguments ',' map_arg */ +#line 3343 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].list)); } +#line 27885 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1074: -#line 3350 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1092: /* map_arguments_opt_comma: map_arguments */ +#line 3348 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 27891 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1075: -#line 3351 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NULL; ;} + case 1093: /* map_arguments_opt_comma: map_arguments ',' */ +#line 3349 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 27897 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1076: + case 1094: /* opt_map_arguments_opt_comma: map_arguments_opt_comma */ #line 3354 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.subquerytype) = PG_ANY_SUBLINK; ;} + { (yyval.list) = (yyvsp[0].list); } +#line 27903 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1077: + case 1095: /* opt_map_arguments_opt_comma: %empty */ #line 3355 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.subquerytype) = PG_ANY_SUBLINK; ;} + { (yyval.list) = NULL; } +#line 27909 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1078: -#line 3356 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.subquerytype) = PG_ALL_SUBLINK; ;} + case 1096: /* sub_type: ANY */ +#line 3358 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.subquerytype) = PG_ANY_SUBLINK; } +#line 27915 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1079: + case 1097: /* sub_type: SOME */ #line 3359 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.subquerytype) = PG_ANY_SUBLINK; } +#line 27921 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1080: + case 1098: /* sub_type: ALL */ #line 3360 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) (yyvsp[(1) - (1)].conststr); ;} + { (yyval.subquerytype) = PG_ALL_SUBLINK; } +#line 27927 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1081: + case 1099: /* all_Op: Op */ #line 3363 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "+"; ;} + { (yyval.str) = (yyvsp[0].str); } +#line 27933 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1082: + case 1100: /* all_Op: MathOp */ #line 3364 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "-"; ;} - break; - - case 1083: -#line 3365 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "*"; ;} - break; - - case 1084: -#line 3366 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "/"; ;} + { (yyval.str) = (char*) (yyvsp[0].conststr); } +#line 27939 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1085: + case 1101: /* MathOp: '+' */ #line 3367 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "//"; ;} + { (yyval.conststr) = "+"; } +#line 27945 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1086: + case 1102: /* MathOp: '-' */ #line 3368 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "%"; ;} + { (yyval.conststr) = "-"; } +#line 27951 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1087: + case 1103: /* MathOp: '*' */ #line 3369 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "^"; ;} + { (yyval.conststr) = "*"; } +#line 27957 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1088: + case 1104: /* MathOp: '/' */ #line 3370 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "**"; ;} + { (yyval.conststr) = "/"; } +#line 27963 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1089: + case 1105: /* MathOp: INTEGER_DIVISION */ #line 3371 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "<"; ;} + { (yyval.conststr) = "//"; } +#line 27969 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1090: + case 1106: /* MathOp: '%' */ #line 3372 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = ">"; ;} + { (yyval.conststr) = "%"; } +#line 27975 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1091: + case 1107: /* MathOp: '^' */ #line 3373 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "="; ;} + { (yyval.conststr) = "^"; } +#line 27981 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1092: + case 1108: /* MathOp: POWER_OF */ #line 3374 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "<="; ;} + { (yyval.conststr) = "**"; } +#line 27987 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1093: + case 1109: /* MathOp: '<' */ #line 3375 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = ">="; ;} + { (yyval.conststr) = "<"; } +#line 27993 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1094: + case 1110: /* MathOp: '>' */ #line 3376 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.conststr) = "<>"; ;} + { (yyval.conststr) = ">"; } +#line 27999 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1095: -#line 3380 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1111: /* MathOp: '=' */ +#line 3377 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.conststr) = "="; } +#line 28005 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1112: /* MathOp: LESS_EQUALS */ +#line 3378 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.conststr) = "<="; } +#line 28011 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1113: /* MathOp: GREATER_EQUALS */ +#line 3379 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.conststr) = ">="; } +#line 28017 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1096: -#line 3382 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + case 1114: /* MathOp: NOT_EQUALS */ +#line 3380 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.conststr) = "<>"; } +#line 28023 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1097: -#line 3387 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1115: /* qual_Op: Op */ +#line 3384 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28029 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1098: -#line 3389 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + case 1116: /* qual_Op: OPERATOR '(' any_operator ')' */ +#line 3386 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28035 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1099: -#line 3394 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1117: /* qual_all_Op: all_Op */ +#line 3391 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28041 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1100: -#line 3396 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + case 1118: /* qual_all_Op: OPERATOR '(' any_operator ')' */ +#line 3393 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28047 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1101: + case 1119: /* subquery_Op: all_Op */ #line 3398 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("~~")); ;} + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28053 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1102: + case 1120: /* subquery_Op: OPERATOR '(' any_operator ')' */ #line 3400 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("!~~")); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 28059 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1103: + case 1121: /* subquery_Op: LIKE */ #line 3402 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("~~~")); ;} + { (yyval.list) = list_make1(makeString("~~")); } +#line 28065 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1104: + case 1122: /* subquery_Op: NOT_LA LIKE */ #line 3404 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("!~~~")); ;} + { (yyval.list) = list_make1(makeString("!~~")); } +#line 28071 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1105: + case 1123: /* subquery_Op: GLOB */ #line 3406 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("~~*")); ;} + { (yyval.list) = list_make1(makeString("~~~")); } +#line 28077 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1106: + case 1124: /* subquery_Op: NOT_LA GLOB */ #line 3408 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString("!~~*")); ;} + { (yyval.list) = list_make1(makeString("!~~~")); } +#line 28083 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1107: -#line 3422 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1125: /* subquery_Op: ILIKE */ +#line 3410 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString("~~*")); } +#line 28089 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1108: -#line 3424 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lcons(makeString((yyvsp[(1) - (3)].str)), (yyvsp[(3) - (3)].list)); ;} + case 1126: /* subquery_Op: NOT_LA ILIKE */ +#line 3412 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString("!~~*")); } +#line 28095 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1109: -#line 3429 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + case 1127: /* any_operator: all_Op */ +#line 3426 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28101 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1128: /* any_operator: ColId '.' any_operator */ +#line 3428 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lcons(makeString((yyvsp[-2].str)), (yyvsp[0].list)); } +#line 28107 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1110: + case 1129: /* c_expr_list: c_expr */ #line 3433 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 28115 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1111: -#line 3440 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (1)].list); - ;} + case 1130: /* c_expr_list: c_expr_list ',' c_expr */ +#line 3437 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 28123 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1112: -#line 3445 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (2)].list); - ;} + case 1131: /* c_expr_list_opt_comma: c_expr_list */ +#line 3444 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = (yyvsp[0].list); + } +#line 28131 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1113: -#line 3451 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + case 1132: /* c_expr_list_opt_comma: c_expr_list ',' */ +#line 3449 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = (yyvsp[-1].list); + } +#line 28139 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1114: + case 1133: /* expr_list: a_expr */ #line 3455 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 28147 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1115: -#line 3462 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (1)].list); - ;} + case 1134: /* expr_list: expr_list ',' a_expr */ +#line 3459 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 28155 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1116: -#line 3467 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (2)].list); - ;} + case 1135: /* expr_list_opt_comma: expr_list */ +#line 3466 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = (yyvsp[0].list); + } +#line 28163 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1117: -#line 3474 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (1)].list); - ;} + case 1136: /* expr_list_opt_comma: expr_list ',' */ +#line 3471 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = (yyvsp[-1].list); + } +#line 28171 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1118: + case 1137: /* opt_expr_list_opt_comma: expr_list_opt_comma */ #line 3478 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = NULL; - ;} + { + (yyval.list) = (yyvsp[0].list); + } +#line 28179 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1119: -#line 3487 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); - ;} + case 1138: /* opt_expr_list_opt_comma: %empty */ +#line 3482 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = NULL; + } +#line 28187 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1120: + case 1139: /* func_arg_list: func_arg_expr */ #line 3491 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].node)); + } +#line 28195 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1121: -#line 3497 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = (yyvsp[(1) - (1)].node); - ;} + case 1140: /* func_arg_list: func_arg_list ',' func_arg_expr */ +#line 3495 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); + } +#line 28203 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1122: + case 1141: /* func_arg_expr: a_expr */ #line 3501 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.node) = (yyvsp[0].node); + } +#line 28211 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1142: /* func_arg_expr: param_name COLON_EQUALS a_expr */ +#line 3505 "third_party/libpg_query/grammar/statements/select.y" + { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); - na->name = (yyvsp[(1) - (3)].str); - na->arg = (PGExpr *) (yyvsp[(3) - (3)].node); + na->name = (yyvsp[-2].str); + na->arg = (PGExpr *) (yyvsp[0].node); na->argnumber = -1; /* until determined */ - na->location = (yylsp[(1) - (3)]); + na->location = (yylsp[-2]); (yyval.node) = (PGNode *) na; - ;} + } +#line 28224 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1123: -#line 3510 "third_party/libpg_query/grammar/statements/select.y" - { + case 1143: /* func_arg_expr: param_name EQUALS_GREATER a_expr */ +#line 3514 "third_party/libpg_query/grammar/statements/select.y" + { PGNamedArgExpr *na = makeNode(PGNamedArgExpr); - na->name = (yyvsp[(1) - (3)].str); - na->arg = (PGExpr *) (yyvsp[(3) - (3)].node); + na->name = (yyvsp[-2].str); + na->arg = (PGExpr *) (yyvsp[0].node); na->argnumber = -1; /* until determined */ - na->location = (yylsp[(1) - (3)]); + na->location = (yylsp[-2]); (yyval.node) = (PGNode *) na; - ;} - break; - - case 1124: -#line 3520 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].typnam)); ;} - break; - - case 1125: -#line 3521 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].typnam)); ;} - break; - - case 1126: -#line 3526 "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 1127: -#line 3529 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + } +#line 28237 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1128: -#line 3536 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 1144: /* type_list: Typename */ +#line 3524 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].typnam)); } +#line 28243 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1129: -#line 3537 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "year"; ;} + case 1145: /* type_list: type_list ',' Typename */ +#line 3525 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].typnam)); } +#line 28249 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1130: -#line 3538 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "month"; ;} + case 1146: /* extract_list: extract_arg FROM a_expr */ +#line 3530 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make2(makeStringConst((yyvsp[-2].str), (yylsp[-2])), (yyvsp[0].node)); + } +#line 28257 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1131: -#line 3539 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "day"; ;} + case 1147: /* extract_list: %empty */ +#line 3533 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 28263 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1132: + case 1148: /* extract_arg: IDENT */ #line 3540 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "hour"; ;} + { (yyval.str) = (yyvsp[0].str); } +#line 28269 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1133: + case 1149: /* extract_arg: year_keyword */ #line 3541 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "minute"; ;} + { (yyval.str) = (char*) "year"; } +#line 28275 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1134: + case 1150: /* extract_arg: month_keyword */ #line 3542 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "second"; ;} + { (yyval.str) = (char*) "month"; } +#line 28281 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1135: + case 1151: /* extract_arg: day_keyword */ #line 3543 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "millisecond"; ;} + { (yyval.str) = (char*) "day"; } +#line 28287 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1136: + case 1152: /* extract_arg: hour_keyword */ #line 3544 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "microsecond"; ;} + { (yyval.str) = (char*) "hour"; } +#line 28293 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1137: + case 1153: /* extract_arg: minute_keyword */ #line 3545 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "week"; ;} + { (yyval.str) = (char*) "minute"; } +#line 28299 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1138: + case 1154: /* extract_arg: second_keyword */ #line 3546 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "decade"; ;} + { (yyval.str) = (char*) "second"; } +#line 28305 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1139: + case 1155: /* extract_arg: millisecond_keyword */ #line 3547 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "century"; ;} + { (yyval.str) = (char*) "millisecond"; } +#line 28311 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1140: + case 1156: /* extract_arg: microsecond_keyword */ #line 3548 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (char*) "millennium"; ;} + { (yyval.str) = (char*) "microsecond"; } +#line 28317 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1141: + case 1157: /* extract_arg: week_keyword */ #line 3549 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (char*) "week"; } +#line 28323 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1142: -#line 3560 "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)); - ;} + case 1158: /* extract_arg: decade_keyword */ +#line 3550 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (char*) "decade"; } +#line 28329 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1159: /* extract_arg: century_keyword */ +#line 3551 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (char*) "century"; } +#line 28335 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1160: /* extract_arg: millennium_keyword */ +#line 3552 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (char*) "millennium"; } +#line 28341 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1161: /* extract_arg: Sconst */ +#line 3553 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 28347 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1143: + case 1162: /* overlay_list: a_expr overlay_placing substr_from substr_for */ #line 3564 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make3((yyvsp[(1) - (3)].node), (yyvsp[(2) - (3)].node), (yyvsp[(3) - (3)].node)); - ;} + { + (yyval.list) = list_make4((yyvsp[-3].node), (yyvsp[-2].node), (yyvsp[-1].node), (yyvsp[0].node)); + } +#line 28355 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1144: -#line 3571 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 1163: /* overlay_list: a_expr overlay_placing substr_from */ +#line 3568 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make3((yyvsp[-2].node), (yyvsp[-1].node), (yyvsp[0].node)); + } +#line 28363 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1145: -#line 3577 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(3) - (3)].node), (yyvsp[(1) - (3)].node)); ;} + case 1164: /* overlay_placing: PLACING a_expr */ +#line 3575 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 28369 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1146: -#line 3578 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1165: /* position_list: b_expr IN_P b_expr */ +#line 3581 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2((yyvsp[0].node), (yyvsp[-2].node)); } +#line 28375 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1147: -#line 3595 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make3((yyvsp[(1) - (3)].node), (yyvsp[(2) - (3)].node), (yyvsp[(3) - (3)].node)); - ;} + case 1166: /* position_list: %empty */ +#line 3582 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 28381 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1148: + case 1167: /* substr_list: a_expr substr_from substr_for */ #line 3599 "third_party/libpg_query/grammar/statements/select.y" - { + { + (yyval.list) = list_make3((yyvsp[-2].node), (yyvsp[-1].node), (yyvsp[0].node)); + } +#line 28389 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1168: /* substr_list: a_expr substr_for substr_from */ +#line 3603 "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)); - ;} + (yyval.list) = list_make3((yyvsp[-2].node), (yyvsp[0].node), (yyvsp[-1].node)); + } +#line 28398 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1149: -#line 3604 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = list_make2((yyvsp[(1) - (2)].node), (yyvsp[(2) - (2)].node)); - ;} + case 1169: /* substr_list: a_expr substr_from */ +#line 3608 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = list_make2((yyvsp[-1].node), (yyvsp[0].node)); + } +#line 28406 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1150: -#line 3608 "third_party/libpg_query/grammar/statements/select.y" - { + case 1170: /* substr_list: a_expr substr_for */ +#line 3612 "third_party/libpg_query/grammar/statements/select.y" + { /* * Since there are no cases where this syntax allows * a textual FOR value, we forcibly cast the argument @@ -27557,523 +28417,599 @@ YYLTYPE yylloc; * which it is likely to do if the second argument * is unknown or doesn't have an implicit cast to int4. */ - (yyval.list) = list_make3((yyvsp[(1) - (2)].node), makeIntConst(1, -1), - makeTypeCast((yyvsp[(2) - (2)].node), + (yyval.list) = list_make3((yyvsp[-1].node), makeIntConst(1, -1), + makeTypeCast((yyvsp[0].node), SystemTypeName("int4"), 0, -1)); - ;} + } +#line 28425 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1151: -#line 3623 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = (yyvsp[(1) - (1)].list); - ;} - break; - - case 1152: + case 1171: /* substr_list: expr_list */ #line 3627 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + { + (yyval.list) = (yyvsp[0].list); + } +#line 28433 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1153: + case 1172: /* substr_list: %empty */ #line 3631 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + { (yyval.list) = NIL; } +#line 28439 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1154: -#line 3634 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + case 1173: /* substr_from: FROM a_expr */ +#line 3635 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 28445 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1155: -#line 3637 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(3) - (3)].list), (yyvsp[(1) - (3)].node)); ;} + case 1174: /* substr_for: FOR a_expr */ +#line 3638 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 28451 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1156: -#line 3638 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + case 1175: /* trim_list: a_expr FROM expr_list_opt_comma */ +#line 3641 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[0].list), (yyvsp[-2].node)); } +#line 28457 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1157: -#line 3639 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1176: /* trim_list: FROM expr_list_opt_comma */ +#line 3642 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 28463 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1158: + case 1177: /* trim_list: expr_list_opt_comma */ #line 3643 "third_party/libpg_query/grammar/statements/select.y" - { + { (yyval.list) = (yyvsp[0].list); } +#line 28469 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1178: /* in_expr: select_with_parens */ +#line 3647 "third_party/libpg_query/grammar/statements/select.y" + { PGSubLink *n = makeNode(PGSubLink); - n->subselect = (yyvsp[(1) - (1)].node); + n->subselect = (yyvsp[0].node); /* other fields will be filled later */ (yyval.node) = (PGNode *)n; - ;} + } +#line 28480 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1159: -#line 3649 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (PGNode *)(yyvsp[(2) - (3)].list); ;} + case 1179: /* in_expr: '(' expr_list_opt_comma ')' */ +#line 3653 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (PGNode *)(yyvsp[-1].list); } +#line 28486 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1160: -#line 3660 "third_party/libpg_query/grammar/statements/select.y" - { + case 1180: /* case_expr: CASE case_arg when_clause_list case_default END_P */ +#line 3664 "third_party/libpg_query/grammar/statements/select.y" + { PGCaseExpr *c = makeNode(PGCaseExpr); c->casetype = InvalidOid; /* not analyzed yet */ - c->arg = (PGExpr *) (yyvsp[(2) - (5)].node); - c->args = (yyvsp[(3) - (5)].list); - c->defresult = (PGExpr *) (yyvsp[(4) - (5)].node); - c->location = (yylsp[(1) - (5)]); + c->arg = (PGExpr *) (yyvsp[-3].node); + c->args = (yyvsp[-2].list); + c->defresult = (PGExpr *) (yyvsp[-1].node); + c->location = (yylsp[-4]); (yyval.node) = (PGNode *)c; - ;} + } +#line 28500 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1161: -#line 3673 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + case 1181: /* when_clause_list: when_clause */ +#line 3677 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 28506 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1162: -#line 3674 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + case 1182: /* when_clause_list: when_clause_list when_clause */ +#line 3678 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 28512 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1163: -#line 3679 "third_party/libpg_query/grammar/statements/select.y" - { + case 1183: /* when_clause: WHEN a_expr THEN a_expr */ +#line 3683 "third_party/libpg_query/grammar/statements/select.y" + { PGCaseWhen *w = makeNode(PGCaseWhen); - w->expr = (PGExpr *) (yyvsp[(2) - (4)].node); - w->result = (PGExpr *) (yyvsp[(4) - (4)].node); - w->location = (yylsp[(1) - (4)]); + w->expr = (PGExpr *) (yyvsp[-2].node); + w->result = (PGExpr *) (yyvsp[0].node); + w->location = (yylsp[-3]); (yyval.node) = (PGNode *)w; - ;} - break; - - case 1164: -#line 3689 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + } +#line 28524 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1165: -#line 3690 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} - break; - - case 1166: + case 1184: /* case_default: ELSE a_expr */ #line 3693 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 28530 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1167: + case 1185: /* case_default: %empty */ #line 3694 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 28536 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1186: /* case_arg: a_expr */ +#line 3697 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 28542 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1168: + case 1187: /* case_arg: %empty */ #line 3698 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeColumnRef((yyvsp[(1) - (1)].str), NIL, (yylsp[(1) - (1)]), yyscanner); - ;} + { (yyval.node) = NULL; } +#line 28548 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1169: + case 1188: /* columnref: ColId */ #line 3702 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeColumnRef((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].list), (yylsp[(1) - (2)]), yyscanner); - ;} + { + (yyval.node) = makeColumnRef((yyvsp[0].str), NIL, (yylsp[0]), yyscanner); + } +#line 28556 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1170: -#line 3709 "third_party/libpg_query/grammar/statements/select.y" - { + case 1189: /* columnref: ColId indirection */ +#line 3706 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeColumnRef((yyvsp[-1].str), (yyvsp[0].list), (yylsp[-1]), yyscanner); + } +#line 28564 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1190: /* indirection_el: '[' a_expr ']' */ +#line 3713 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = false; ai->lidx = NULL; - ai->uidx = (yyvsp[(2) - (3)].node); + ai->uidx = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28576 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1171: -#line 3717 "third_party/libpg_query/grammar/statements/select.y" - { + case 1191: /* indirection_el: '[' opt_slice_bound ':' opt_slice_bound ']' */ +#line 3721 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (5)].node); - ai->uidx = (yyvsp[(4) - (5)].node); + ai->lidx = (yyvsp[-3].node); + ai->uidx = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28588 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1172: -#line 3724 "third_party/libpg_query/grammar/statements/select.y" - { + case 1192: /* indirection_el: '[' opt_slice_bound ':' opt_slice_bound ':' opt_slice_bound ']' */ +#line 3728 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (7)].node); - ai->uidx = (yyvsp[(4) - (7)].node); - ai->step = (yyvsp[(6) - (7)].node); + ai->lidx = (yyvsp[-5].node); + ai->uidx = (yyvsp[-3].node); + ai->step = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28601 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1173: -#line 3732 "third_party/libpg_query/grammar/statements/select.y" - { + case 1193: /* indirection_el: '[' opt_slice_bound ':' '-' ':' opt_slice_bound ']' */ +#line 3736 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (7)].node); - ai->step = (yyvsp[(6) - (7)].node); + ai->lidx = (yyvsp[-5].node); + ai->step = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28613 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1174: -#line 3742 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + case 1194: /* opt_slice_bound: a_expr */ +#line 3746 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = (yyvsp[0].node); } +#line 28619 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1175: -#line 3743 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.node) = NULL; ;} + case 1195: /* opt_slice_bound: %empty */ +#line 3747 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.node) = NULL; } +#line 28625 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1176: -#line 3748 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1196: /* opt_indirection: %empty */ +#line 3752 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 28631 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1177: -#line 3749 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + case 1197: /* opt_indirection: opt_indirection indirection_el */ +#line 3753 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 28637 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1178: -#line 3753 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NULL; ;} + case 1198: /* opt_func_arguments: %empty */ +#line 3757 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NULL; } +#line 28643 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1179: -#line 3754 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(NULL); ;} + case 1199: /* opt_func_arguments: '(' ')' */ +#line 3758 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(NULL); } +#line 28649 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1180: -#line 3755 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + case 1200: /* opt_func_arguments: '(' func_arg_list ')' */ +#line 3759 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28655 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1181: -#line 3760 "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)])); + case 1201: /* extended_indirection_el: '.' attr_name opt_func_arguments */ +#line 3764 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].list)) { + PGFuncCall *n = makeFuncCall(list_make1(makeString((yyvsp[-1].str))), (yyvsp[0].list)->head->data.ptr_value ? (yyvsp[0].list) : NULL, (yylsp[-1])); (yyval.node) = (PGNode *) n; } else { - (yyval.node) = (PGNode *) makeString((yyvsp[(2) - (3)].str)); + (yyval.node) = (PGNode *) makeString((yyvsp[-1].str)); } - ;} + } +#line 28668 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1182: -#line 3769 "third_party/libpg_query/grammar/statements/select.y" - { + case 1202: /* extended_indirection_el: '[' a_expr ']' */ +#line 3773 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = false; ai->lidx = NULL; - ai->uidx = (yyvsp[(2) - (3)].node); + ai->uidx = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28680 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1183: -#line 3777 "third_party/libpg_query/grammar/statements/select.y" - { + case 1203: /* extended_indirection_el: '[' opt_slice_bound ':' opt_slice_bound ']' */ +#line 3781 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (5)].node); - ai->uidx = (yyvsp[(4) - (5)].node); + ai->lidx = (yyvsp[-3].node); + ai->uidx = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28692 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1184: -#line 3784 "third_party/libpg_query/grammar/statements/select.y" - { + case 1204: /* extended_indirection_el: '[' opt_slice_bound ':' opt_slice_bound ':' opt_slice_bound ']' */ +#line 3788 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (7)].node); - ai->uidx = (yyvsp[(4) - (7)].node); - ai->step = (yyvsp[(6) - (7)].node); + ai->lidx = (yyvsp[-5].node); + ai->uidx = (yyvsp[-3].node); + ai->step = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} + } +#line 28705 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1185: -#line 3793 "third_party/libpg_query/grammar/statements/select.y" - { + case 1205: /* extended_indirection_el: '[' opt_slice_bound ':' '-' ':' opt_slice_bound ']' */ +#line 3797 "third_party/libpg_query/grammar/statements/select.y" + { PGAIndices *ai = makeNode(PGAIndices); ai->is_slice = true; - ai->lidx = (yyvsp[(2) - (7)].node); - ai->step = (yyvsp[(6) - (7)].node); + ai->lidx = (yyvsp[-5].node); + ai->step = (yyvsp[-1].node); (yyval.node) = (PGNode *) ai; - ;} - break; - - case 1186: -#line 3808 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + } +#line 28717 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1187: -#line 3809 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); ;} + case 1206: /* opt_extended_indirection: %empty */ +#line 3812 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 28723 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1190: -#line 3825 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1207: /* opt_extended_indirection: opt_extended_indirection extended_indirection_el */ +#line 3813 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); } +#line 28729 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1191: -#line 3826 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1210: /* opt_target_list_opt_comma: target_list_opt_comma */ +#line 3829 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 28735 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1192: + case 1211: /* opt_target_list_opt_comma: %empty */ #line 3830 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].target)); ;} + { (yyval.list) = NIL; } +#line 28741 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1193: -#line 3831 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].target)); ;} + case 1212: /* target_list: target_el */ +#line 3834 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].target)); } +#line 28747 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1194: + case 1213: /* target_list: target_list ',' target_el */ #line 3835 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].target)); } +#line 28753 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1195: -#line 3836 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 1214: /* target_list_opt_comma: target_list */ +#line 3839 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 28759 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1196: + case 1215: /* target_list_opt_comma: target_list ',' */ #line 3840 "third_party/libpg_query/grammar/statements/select.y" - { + { (yyval.list) = (yyvsp[-1].list); } +#line 28765 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1216: /* target_el: a_expr AS ColLabelOrString */ +#line 3844 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.target) = makeNode(PGResTarget); - (yyval.target)->name = (yyvsp[(3) - (3)].str); + (yyval.target)->name = (yyvsp[0].str); (yyval.target)->indirection = NIL; - (yyval.target)->val = (PGNode *)(yyvsp[(1) - (3)].node); - (yyval.target)->location = (yylsp[(1) - (3)]); - ;} + (yyval.target)->val = (PGNode *)(yyvsp[-2].node); + (yyval.target)->location = (yylsp[-2]); + } +#line 28777 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1197: -#line 3856 "third_party/libpg_query/grammar/statements/select.y" - { + case 1217: /* target_el: a_expr IDENT */ +#line 3860 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.target) = makeNode(PGResTarget); - (yyval.target)->name = (yyvsp[(2) - (2)].str); + (yyval.target)->name = (yyvsp[0].str); (yyval.target)->indirection = NIL; - (yyval.target)->val = (PGNode *)(yyvsp[(1) - (2)].node); - (yyval.target)->location = (yylsp[(1) - (2)]); - ;} + (yyval.target)->val = (PGNode *)(yyvsp[-1].node); + (yyval.target)->location = (yylsp[-1]); + } +#line 28789 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1198: -#line 3864 "third_party/libpg_query/grammar/statements/select.y" - { + case 1218: /* target_el: a_expr */ +#line 3868 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.target) = makeNode(PGResTarget); (yyval.target)->name = NULL; (yyval.target)->indirection = NIL; - (yyval.target)->val = (PGNode *)(yyvsp[(1) - (1)].node); - (yyval.target)->location = (yylsp[(1) - (1)]); - ;} - break; - - case 1199: -#line 3873 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + (yyval.target)->val = (PGNode *)(yyvsp[0].node); + (yyval.target)->location = (yylsp[0]); + } +#line 28801 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1200: -#line 3874 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(2) - (2)].str))); ;} - break; - - case 1201: + case 1219: /* except_list: EXCLUDE '(' name_list_opt_comma ')' */ #line 3877 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 28807 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1202: + case 1220: /* except_list: EXCLUDE ColId */ #line 3878 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NULL; ;} + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28813 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1203: + case 1221: /* opt_except_list: except_list */ #line 3881 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make2((yyvsp[(1) - (3)].node), makeString((yyvsp[(3) - (3)].str))); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 28819 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1204: -#line 3885 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].list)); ;} + case 1222: /* opt_except_list: %empty */ +#line 3882 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NULL; } +#line 28825 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1205: -#line 3886 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].list)); ;} + case 1223: /* replace_list_el: a_expr AS ColId */ +#line 3885 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make2((yyvsp[-2].node), makeString((yyvsp[0].str))); } +#line 28831 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1206: -#line 3890 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1224: /* replace_list: replace_list_el */ +#line 3889 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].list)); } +#line 28837 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1207: -#line 3891 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 1225: /* replace_list: replace_list ',' replace_list_el */ +#line 3890 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].list)); } +#line 28843 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1208: + case 1226: /* replace_list_opt_comma: replace_list */ #line 3894 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(3) - (4)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 28849 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1209: + case 1227: /* replace_list_opt_comma: replace_list ',' */ #line 3895 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(2) - (2)].list)); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 28855 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1210: -#line 3896 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NULL; ;} + case 1228: /* opt_replace_list: REPLACE '(' replace_list_opt_comma ')' */ +#line 3898 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28861 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1211: -#line 3906 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].range)); ;} + case 1229: /* opt_replace_list: REPLACE replace_list_el */ +#line 3899 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].list)); } +#line 28867 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1212: -#line 3907 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].range)); ;} + case 1230: /* opt_replace_list: %empty */ +#line 3900 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NULL; } +#line 28873 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1213: -#line 3912 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1231: /* qualified_name_list: qualified_name */ +#line 3910 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1((yyvsp[0].range)); } +#line 28879 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1214: -#line 3914 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeString((yyvsp[(3) - (3)].str))); ;} + case 1232: /* qualified_name_list: qualified_name_list ',' qualified_name */ +#line 3911 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].range)); } +#line 28885 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1215: -#line 3919 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1233: /* name_list: name */ +#line 3916 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28891 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1216: -#line 3920 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (2)].list); ;} + case 1234: /* name_list: name_list ',' name */ +#line 3918 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = lappend((yyvsp[-2].list), makeString((yyvsp[0].str))); } +#line 28897 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1217: -#line 3924 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(1) - (1)].list); ;} + case 1235: /* name_list_opt_comma: name_list */ +#line 3923 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[0].list); } +#line 28903 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1218: -#line 3925 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + case 1236: /* name_list_opt_comma: name_list ',' */ +#line 3924 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28909 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1219: + case 1237: /* name_list_opt_comma_opt_bracket: name_list_opt_comma */ #line 3928 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 28915 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1220: -#line 3940 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + case 1238: /* name_list_opt_comma_opt_bracket: '(' name_list_opt_comma ')' */ +#line 3929 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 28921 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1221: -#line 3943 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.list) = check_func_name(lcons(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].list)), - yyscanner); - ;} + case 1239: /* name: ColIdOrString */ +#line 3932 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 28927 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1222: -#line 3954 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeIntConst((yyvsp[(1) - (1)].ival), (yylsp[(1) - (1)])); - ;} + case 1240: /* func_name: function_name_token */ +#line 3944 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 28933 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1241: /* func_name: ColId indirection */ +#line 3947 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.list) = check_func_name(lcons(makeString((yyvsp[-1].str)), (yyvsp[0].list)), + yyscanner); + } +#line 28942 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1223: + case 1242: /* AexprConst: Iconst */ #line 3958 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeFloatConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + { + (yyval.node) = makeIntConst((yyvsp[0].ival), (yylsp[0])); + } +#line 28950 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1224: + case 1243: /* AexprConst: FCONST */ #line 3962 "third_party/libpg_query/grammar/statements/select.y" - { - if ((yyvsp[(2) - (2)].list)) + { + (yyval.node) = makeFloatConst((yyvsp[0].str), (yylsp[0])); + } +#line 28958 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1244: /* AexprConst: Sconst opt_indirection */ +#line 3966 "third_party/libpg_query/grammar/statements/select.y" + { + if ((yyvsp[0].list)) { PGAIndirection *n = makeNode(PGAIndirection); - n->arg = makeStringConst((yyvsp[(1) - (2)].str), (yylsp[(1) - (2)])); - n->indirection = check_indirection((yyvsp[(2) - (2)].list), yyscanner); + n->arg = makeStringConst((yyvsp[-1].str), (yylsp[-1])); + n->indirection = check_indirection((yyvsp[0].list), yyscanner); (yyval.node) = (PGNode *) n; } else - (yyval.node) = makeStringConst((yyvsp[(1) - (2)].str), (yylsp[(1) - (2)])); - ;} + (yyval.node) = makeStringConst((yyvsp[-1].str), (yylsp[-1])); + } +#line 28974 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1225: -#line 3974 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeBitStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + case 1245: /* AexprConst: BCONST */ +#line 3978 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeBitStringConst((yyvsp[0].str), (yylsp[0])); + } +#line 28982 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1226: -#line 3978 "third_party/libpg_query/grammar/statements/select.y" - { + case 1246: /* AexprConst: XCONST */ +#line 3982 "third_party/libpg_query/grammar/statements/select.y" + { /* This is a bit constant per SQL99: * Without Feature F511, "BIT data type", * a shall not be a * or a . */ - (yyval.node) = makeBitStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + (yyval.node) = makeBitStringConst((yyvsp[0].str), (yylsp[0])); + } +#line 28995 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1227: -#line 3987 "third_party/libpg_query/grammar/statements/select.y" - { + case 1247: /* AexprConst: func_name Sconst */ +#line 3991 "third_party/libpg_query/grammar/statements/select.y" + { /* generic type 'literal' syntax */ - PGTypeName *t = makeTypeNameFromNameList((yyvsp[(1) - (2)].list)); - t->location = (yylsp[(1) - (2)]); - (yyval.node) = makeStringConstCast((yyvsp[(2) - (2)].str), (yylsp[(2) - (2)]), t); - ;} + PGTypeName *t = makeTypeNameFromNameList((yyvsp[-1].list)); + t->location = (yylsp[-1]); + (yyval.node) = makeStringConstCast((yyvsp[0].str), (yylsp[0]), t); + } +#line 29006 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1228: -#line 3994 "third_party/libpg_query/grammar/statements/select.y" - { + case 1248: /* AexprConst: func_name '(' func_arg_list opt_sort_clause opt_ignore_nulls ')' Sconst */ +#line 3998 "third_party/libpg_query/grammar/statements/select.y" + { /* generic syntax with a type modifier */ - PGTypeName *t = makeTypeNameFromNameList((yyvsp[(1) - (7)].list)); + PGTypeName *t = makeTypeNameFromNameList((yyvsp[-6].list)); PGListCell *lc; /* @@ -28082,7 +29018,7 @@ YYLTYPE yylloc; * don't actually wish to allow PGNamedArgExpr in this * context, ORDER BY, nor IGNORE NULLS. */ - foreach(lc, (yyvsp[(3) - (7)].list)) + foreach(lc, (yyvsp[-4].list)) { PGNamedArgExpr *arg = (PGNamedArgExpr *) lfirst(lc); @@ -28092,281 +29028,315 @@ YYLTYPE yylloc; errmsg("type modifier cannot have parameter name"), parser_errposition(arg->location))); } - if ((yyvsp[(4) - (7)].list) != NIL) + if ((yyvsp[-3].list) != NIL) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("type modifier cannot have ORDER BY"), - parser_errposition((yylsp[(4) - (7)])))); - if ((yyvsp[(5) - (7)].boolean) != false) + parser_errposition((yylsp[-3])))); + if ((yyvsp[-2].boolean) != false) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("type modifier cannot have IGNORE NULLS"), - parser_errposition((yylsp[(5) - (7)])))); - + parser_errposition((yylsp[-2])))); - t->typmods = (yyvsp[(3) - (7)].list); - t->location = (yylsp[(1) - (7)]); - (yyval.node) = makeStringConstCast((yyvsp[(7) - (7)].str), (yylsp[(7) - (7)]), t); - ;} - break; - case 1229: -#line 4032 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeStringConstCast((yyvsp[(2) - (2)].str), (yylsp[(2) - (2)]), (yyvsp[(1) - (2)].typnam)); - ;} + t->typmods = (yyvsp[-4].list); + t->location = (yylsp[-6]); + (yyval.node) = makeStringConstCast((yyvsp[0].str), (yylsp[0]), t); + } +#line 29048 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1230: + case 1249: /* AexprConst: ConstTypename Sconst */ #line 4036 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeIntervalNode((yyvsp[(3) - (5)].node), (yylsp[(3) - (5)]), (yyvsp[(5) - (5)].list)); - ;} + { + (yyval.node) = makeStringConstCast((yyvsp[0].str), (yylsp[0]), (yyvsp[-1].typnam)); + } +#line 29056 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1231: + case 1250: /* AexprConst: ConstInterval '(' a_expr ')' opt_interval */ #line 4040 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeIntervalNode((yyvsp[(2) - (3)].ival), (yylsp[(2) - (3)]), (yyvsp[(3) - (3)].list)); - ;} + { + (yyval.node) = makeIntervalNode((yyvsp[-2].node), (yylsp[-2]), (yyvsp[0].list)); + } +#line 29064 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1232: + case 1251: /* AexprConst: ConstInterval Iconst opt_interval */ #line 4044 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeIntervalNode((yyvsp[(2) - (3)].str), (yylsp[(2) - (3)]), (yyvsp[(3) - (3)].list)); - ;} + { + (yyval.node) = makeIntervalNode((yyvsp[-1].ival), (yylsp[-1]), (yyvsp[0].list)); + } +#line 29072 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1233: + case 1252: /* AexprConst: ConstInterval Sconst opt_interval */ #line 4048 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeBoolAConst(true, (yylsp[(1) - (1)])); - ;} + { + (yyval.node) = makeIntervalNode((yyvsp[-1].str), (yylsp[-1]), (yyvsp[0].list)); + } +#line 29080 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1234: + case 1253: /* AexprConst: TRUE_P */ #line 4052 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeBoolAConst(false, (yylsp[(1) - (1)])); - ;} + { + (yyval.node) = makeBoolAConst(true, (yylsp[0])); + } +#line 29088 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1235: + case 1254: /* AexprConst: FALSE_P */ #line 4056 "third_party/libpg_query/grammar/statements/select.y" - { - (yyval.node) = makeNullAConst((yylsp[(1) - (1)])); - ;} - break; - - case 1236: -#line 4061 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.ival) = (yyvsp[(1) - (1)].ival); ;} + { + (yyval.node) = makeBoolAConst(false, (yylsp[0])); + } +#line 29096 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1237: -#line 4078 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 1255: /* AexprConst: NULL_P */ +#line 4060 "third_party/libpg_query/grammar/statements/select.y" + { + (yyval.node) = makeNullAConst((yylsp[0])); + } +#line 29104 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1238: -#line 4079 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + case 1256: /* Iconst: ICONST */ +#line 4065 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.ival) = (yyvsp[0].ival); } +#line 29110 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1239: -#line 4080 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + case 1257: /* type_function_name: IDENT */ +#line 4082 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 29116 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1240: + case 1258: /* type_function_name: unreserved_keyword */ #line 4083 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29122 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1241: + case 1259: /* type_function_name: type_func_name_keyword */ #line 4084 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29128 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1242: -#line 4085 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + case 1260: /* function_name_token: IDENT */ +#line 4087 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 29134 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1243: + case 1261: /* function_name_token: unreserved_keyword */ #line 4088 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29140 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1244: + case 1262: /* function_name_token: func_name_keyword */ #line 4089 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29146 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1245: -#line 4090 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + case 1263: /* type_name_token: IDENT */ +#line 4092 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 29152 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1246: + case 1264: /* type_name_token: unreserved_keyword */ #line 4093 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(1) - (1)].str))); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29158 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1247: + case 1265: /* type_name_token: type_name_keyword */ #line 4094 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lcons(makeString((yyvsp[(1) - (2)].str)), (yyvsp[(2) - (2)].list)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29164 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1248: + case 1266: /* any_name: ColId */ +#line 4097 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 29170 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1267: /* any_name: ColId attrs */ #line 4098 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = list_make1(makeString((yyvsp[(2) - (2)].str))); ;} + { (yyval.list) = lcons(makeString((yyvsp[-1].str)), (yyvsp[0].list)); } +#line 29176 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1249: -#line 4100 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), makeString((yyvsp[(3) - (3)].str))); ;} + case 1268: /* attrs: '.' attr_name */ +#line 4102 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = list_make1(makeString((yyvsp[0].str))); } +#line 29182 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1250: + case 1269: /* attrs: attrs '.' attr_name */ #line 4104 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = lappend((yyvsp[-2].list), makeString((yyvsp[0].str))); } +#line 29188 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1270: /* opt_name_list: '(' name_list_opt_comma ')' */ +#line 4108 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = (yyvsp[-1].list); } +#line 29194 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1251: -#line 4105 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.list) = NIL; ;} + case 1271: /* opt_name_list: %empty */ +#line 4109 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.list) = NIL; } +#line 29200 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1253: -#line 4112 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 1273: /* ColLabelOrString: ColLabel */ +#line 4116 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 29206 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1254: -#line 4113 "third_party/libpg_query/grammar/statements/select.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + case 1274: /* ColLabelOrString: SCONST */ +#line 4117 "third_party/libpg_query/grammar/statements/select.y" + { (yyval.str) = (yyvsp[0].str); } +#line 29212 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1255: + case 1275: /* PrepareStmt: PREPARE name prep_type_clause AS PreparableStmt */ #line 8 "third_party/libpg_query/grammar/statements/prepare.y" - { + { PGPrepareStmt *n = makeNode(PGPrepareStmt); - n->name = (yyvsp[(2) - (5)].str); - n->argtypes = (yyvsp[(3) - (5)].list); - n->query = (yyvsp[(5) - (5)].node); + n->name = (yyvsp[-3].str); + n->argtypes = (yyvsp[-2].list); + n->query = (yyvsp[0].node); (yyval.node) = (PGNode *) n; - ;} + } +#line 29224 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1256: + case 1276: /* prep_type_clause: '(' type_list ')' */ #line 18 "third_party/libpg_query/grammar/statements/prepare.y" - { (yyval.list) = (yyvsp[(2) - (3)].list); ;} + { (yyval.list) = (yyvsp[-1].list); } +#line 29230 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1257: + case 1277: /* prep_type_clause: %empty */ #line 19 "third_party/libpg_query/grammar/statements/prepare.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 29236 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1263: + case 1284: /* CreateSchemaStmt: CREATE_P SCHEMA qualified_name OptSchemaEltList */ #line 8 "third_party/libpg_query/grammar/statements/create_schema.y" - { + { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); - if ((yyvsp[(3) - (4)].range)->catalogname) { + if ((yyvsp[-1].range)->catalogname) { ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("CREATE SCHEMA too many dots: expected \"catalog.schema\" or \"schema\""), - parser_errposition((yylsp[(3) - (4)])))); + parser_errposition((yylsp[-1])))); } - if ((yyvsp[(3) - (4)].range)->schemaname) { - n->catalogname = (yyvsp[(3) - (4)].range)->schemaname; - n->schemaname = (yyvsp[(3) - (4)].range)->relname; + if ((yyvsp[-1].range)->schemaname) { + n->catalogname = (yyvsp[-1].range)->schemaname; + n->schemaname = (yyvsp[-1].range)->relname; } else { - n->schemaname = (yyvsp[(3) - (4)].range)->relname; + n->schemaname = (yyvsp[-1].range)->relname; } - n->schemaElts = (yyvsp[(4) - (4)].list); + n->schemaElts = (yyvsp[0].list); n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 29259 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1264: + case 1285: /* CreateSchemaStmt: CREATE_P SCHEMA IF_P NOT EXISTS qualified_name OptSchemaEltList */ #line 27 "third_party/libpg_query/grammar/statements/create_schema.y" - { + { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); - if ((yyvsp[(6) - (7)].range)->catalogname) { + if ((yyvsp[-1].range)->catalogname) { ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("CREATE SCHEMA too many dots: expected \"catalog.schema\" or \"schema\""), - parser_errposition((yylsp[(6) - (7)])))); + parser_errposition((yylsp[-1])))); } - if ((yyvsp[(6) - (7)].range)->schemaname) { - n->catalogname = (yyvsp[(6) - (7)].range)->schemaname; - n->schemaname = (yyvsp[(6) - (7)].range)->relname; + if ((yyvsp[-1].range)->schemaname) { + n->catalogname = (yyvsp[-1].range)->schemaname; + n->schemaname = (yyvsp[-1].range)->relname; } else { - n->schemaname = (yyvsp[(6) - (7)].range)->relname; + n->schemaname = (yyvsp[-1].range)->relname; } - if ((yyvsp[(7) - (7)].list) != NIL) + if ((yyvsp[0].list) != NIL) ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("CREATE SCHEMA IF NOT EXISTS cannot include schema elements"), - parser_errposition((yylsp[(7) - (7)])))); - n->schemaElts = (yyvsp[(7) - (7)].list); + parser_errposition((yylsp[0])))); + n->schemaElts = (yyvsp[0].list); n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 29287 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1265: + case 1286: /* CreateSchemaStmt: CREATE_P OR REPLACE SCHEMA qualified_name OptSchemaEltList */ #line 51 "third_party/libpg_query/grammar/statements/create_schema.y" - { + { PGCreateSchemaStmt *n = makeNode(PGCreateSchemaStmt); - if ((yyvsp[(5) - (6)].range)->catalogname) { + if ((yyvsp[-1].range)->catalogname) { ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("CREATE SCHEMA too many dots: expected \"catalog.schema\" or \"schema\""), - parser_errposition((yylsp[(5) - (6)])))); + parser_errposition((yylsp[-1])))); } - if ((yyvsp[(5) - (6)].range)->schemaname) { - n->catalogname = (yyvsp[(5) - (6)].range)->schemaname; - n->schemaname = (yyvsp[(5) - (6)].range)->relname; + if ((yyvsp[-1].range)->schemaname) { + n->catalogname = (yyvsp[-1].range)->schemaname; + n->schemaname = (yyvsp[-1].range)->relname; } else { - n->schemaname = (yyvsp[(5) - (6)].range)->relname; + n->schemaname = (yyvsp[-1].range)->relname; } - n->schemaElts = (yyvsp[(6) - (6)].list); + n->schemaElts = (yyvsp[0].list); n->onconflict = PG_REPLACE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 29310 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1266: + case 1287: /* OptSchemaEltList: OptSchemaEltList schema_stmt */ #line 74 "third_party/libpg_query/grammar/statements/create_schema.y" - { + { if ((yyloc) < 0) /* see comments for YYLLOC_DEFAULT */ - (yyloc) = (yylsp[(2) - (2)]); - (yyval.list) = lappend((yyvsp[(1) - (2)].list), (yyvsp[(2) - (2)].node)); - ;} + (yyloc) = (yylsp[0]); + (yyval.list) = lappend((yyvsp[-1].list), (yyvsp[0].node)); + } +#line 29320 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1267: + case 1288: /* OptSchemaEltList: %empty */ #line 80 "third_party/libpg_query/grammar/statements/create_schema.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 29326 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1272: + case 1293: /* IndexStmt: CREATE_P opt_unique INDEX opt_concurrently opt_index_name ON qualified_name access_method_clause '(' index_params ')' opt_reloptions where_clause */ #line 11 "third_party/libpg_query/grammar/statements/index.y" - { + { PGIndexStmt *n = makeNode(PGIndexStmt); - n->unique = (yyvsp[(2) - (13)].boolean); - n->concurrent = (yyvsp[(4) - (13)].boolean); - n->idxname = (yyvsp[(5) - (13)].str); - n->relation = (yyvsp[(7) - (13)].range); - n->accessMethod = (yyvsp[(8) - (13)].str); - n->indexParams = (yyvsp[(10) - (13)].list); - n->options = (yyvsp[(12) - (13)].list); - n->whereClause = (yyvsp[(13) - (13)].node); + n->unique = (yyvsp[-11].boolean); + n->concurrent = (yyvsp[-9].boolean); + n->idxname = (yyvsp[-8].str); + n->relation = (yyvsp[-6].range); + n->accessMethod = (yyvsp[-5].str); + n->indexParams = (yyvsp[-3].list); + n->options = (yyvsp[-1].list); + n->whereClause = (yyvsp[0].node); n->excludeOpNames = NIL; n->idxcomment = NULL; n->indexOid = InvalidOid; @@ -28378,21 +29348,22 @@ YYLTYPE yylloc; n->transformed = false; n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 29353 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1273: + case 1294: /* IndexStmt: CREATE_P opt_unique INDEX opt_concurrently IF_P NOT EXISTS index_name ON qualified_name access_method_clause '(' index_params ')' opt_reloptions where_clause */ #line 36 "third_party/libpg_query/grammar/statements/index.y" - { + { PGIndexStmt *n = makeNode(PGIndexStmt); - n->unique = (yyvsp[(2) - (16)].boolean); - n->concurrent = (yyvsp[(4) - (16)].boolean); - n->idxname = (yyvsp[(8) - (16)].str); - n->relation = (yyvsp[(10) - (16)].range); - n->accessMethod = (yyvsp[(11) - (16)].str); - n->indexParams = (yyvsp[(13) - (16)].list); - n->options = (yyvsp[(15) - (16)].list); - n->whereClause = (yyvsp[(16) - (16)].node); + n->unique = (yyvsp[-14].boolean); + n->concurrent = (yyvsp[-12].boolean); + n->idxname = (yyvsp[-8].str); + n->relation = (yyvsp[-6].range); + n->accessMethod = (yyvsp[-5].str); + n->indexParams = (yyvsp[-3].list); + n->options = (yyvsp[-1].list); + n->whereClause = (yyvsp[0].node); n->excludeOpNames = NIL; n->idxcomment = NULL; n->indexOid = InvalidOid; @@ -28404,1365 +29375,1601 @@ YYLTYPE yylloc; n->transformed = false; n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 29380 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1274: + case 1295: /* access_method: ColId */ #line 62 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29386 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1275: + case 1296: /* access_method_clause: USING access_method */ #line 66 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.str) = (yyvsp[(2) - (2)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29392 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1276: + case 1297: /* access_method_clause: %empty */ #line 67 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.str) = (char*) DEFAULT_INDEX_TYPE; ;} + { (yyval.str) = (char*) DEFAULT_INDEX_TYPE; } +#line 29398 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1277: + case 1298: /* opt_concurrently: CONCURRENTLY */ #line 72 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 29404 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1278: + case 1299: /* opt_concurrently: %empty */ #line 73 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 29410 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1279: + case 1300: /* opt_index_name: index_name */ #line 78 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29416 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1280: + case 1301: /* opt_index_name: %empty */ #line 79 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.str) = NULL; ;} + { (yyval.str) = NULL; } +#line 29422 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1281: + case 1302: /* opt_reloptions: WITH reloptions */ #line 83 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 29428 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1282: + case 1303: /* opt_reloptions: %empty */ #line 84 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 29434 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1283: + case 1304: /* opt_unique: UNIQUE */ #line 89 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 29440 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1284: + case 1305: /* opt_unique: %empty */ #line 90 "third_party/libpg_query/grammar/statements/index.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 29446 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1285: + case 1306: /* AlterObjectSchemaStmt: ALTER TABLE relation_expr SET SCHEMA name */ #line 8 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(3) - (6)].range); - n->newschema = (yyvsp[(6) - (6)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 29459 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1286: + case 1307: /* AlterObjectSchemaStmt: ALTER TABLE IF_P EXISTS relation_expr SET SCHEMA name */ #line 17 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_TABLE; - n->relation = (yyvsp[(5) - (8)].range); - n->newschema = (yyvsp[(8) - (8)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 29472 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1287: + case 1308: /* AlterObjectSchemaStmt: ALTER SEQUENCE qualified_name SET SCHEMA name */ #line 26 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_SEQUENCE; - n->relation = (yyvsp[(3) - (6)].range); - n->newschema = (yyvsp[(6) - (6)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 29485 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1288: + case 1309: /* AlterObjectSchemaStmt: ALTER SEQUENCE IF_P EXISTS qualified_name SET SCHEMA name */ #line 35 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_SEQUENCE; - n->relation = (yyvsp[(5) - (8)].range); - n->newschema = (yyvsp[(8) - (8)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 29498 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1289: + case 1310: /* AlterObjectSchemaStmt: ALTER VIEW qualified_name SET SCHEMA name */ #line 44 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_VIEW; - n->relation = (yyvsp[(3) - (6)].range); - n->newschema = (yyvsp[(6) - (6)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = false; (yyval.node) = (PGNode *)n; - ;} + } +#line 29511 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1290: + case 1311: /* AlterObjectSchemaStmt: ALTER VIEW IF_P EXISTS qualified_name SET SCHEMA name */ #line 53 "third_party/libpg_query/grammar/statements/alter_schema.y" - { + { PGAlterObjectSchemaStmt *n = makeNode(PGAlterObjectSchemaStmt); n->objectType = PG_OBJECT_VIEW; - n->relation = (yyvsp[(5) - (8)].range); - n->newschema = (yyvsp[(8) - (8)].str); + n->relation = (yyvsp[-3].range); + n->newschema = (yyvsp[0].str); n->missing_ok = true; (yyval.node) = (PGNode *)n; - ;} + } +#line 29524 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1291: + case 1312: /* CheckPointStmt: FORCE CHECKPOINT opt_col_id */ #line 6 "third_party/libpg_query/grammar/statements/checkpoint.y" - { + { PGCheckPointStmt *n = makeNode(PGCheckPointStmt); n->force = true; - n->name = (yyvsp[(3) - (3)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 29535 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1292: + case 1313: /* CheckPointStmt: CHECKPOINT opt_col_id */ #line 13 "third_party/libpg_query/grammar/statements/checkpoint.y" - { + { PGCheckPointStmt *n = makeNode(PGCheckPointStmt); n->force = false; - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 29546 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1293: + case 1314: /* opt_col_id: ColId */ #line 22 "third_party/libpg_query/grammar/statements/checkpoint.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29552 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1294: + case 1315: /* opt_col_id: %empty */ #line 23 "third_party/libpg_query/grammar/statements/checkpoint.y" - { (yyval.str) = NULL; ;} + { (yyval.str) = NULL; } +#line 29558 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1316: /* CommentOnStmt: COMMENT ON comment_on_type_any_name qualified_name IS comment_value */ +#line 8 "third_party/libpg_query/grammar/statements/comment_on.y" + { + PGCommentOnStmt *n = makeNode(PGCommentOnStmt); + n->object_type = (yyvsp[-3].objtype); + n->name = (yyvsp[-2].range); + n->value = (yyvsp[0].node); + (yyval.node) = (PGNode *)n; + } +#line 29570 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1317: /* CommentOnStmt: COMMENT ON COLUMN a_expr IS comment_value */ +#line 16 "third_party/libpg_query/grammar/statements/comment_on.y" + { + PGCommentOnStmt *n = makeNode(PGCommentOnStmt); + n->object_type = PG_OBJECT_COLUMN; + n->column_expr = (yyvsp[-2].node); + n->value = (yyvsp[0].node); + (yyval.node) = (PGNode *)n; + } +#line 29582 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1318: /* comment_value: Sconst */ +#line 26 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.node) = makeStringConst((yyvsp[0].str), (yylsp[0])); } +#line 29588 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1319: /* comment_value: NULL_P */ +#line 27 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.node) = makeNullAConst((yylsp[0])); } +#line 29594 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1320: /* comment_on_type_any_name: TABLE */ +#line 30 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_TABLE; } +#line 29600 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1321: /* comment_on_type_any_name: SEQUENCE */ +#line 31 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_SEQUENCE; } +#line 29606 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1322: /* comment_on_type_any_name: FUNCTION */ +#line 32 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_FUNCTION; } +#line 29612 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1323: /* comment_on_type_any_name: MACRO */ +#line 33 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_FUNCTION; } +#line 29618 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1324: /* comment_on_type_any_name: MACRO TABLE */ +#line 34 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_TABLE_MACRO; } +#line 29624 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1325: /* comment_on_type_any_name: VIEW */ +#line 35 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_VIEW; } +#line 29630 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1326: /* comment_on_type_any_name: DATABASE */ +#line 36 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_DATABASE; } +#line 29636 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1327: /* comment_on_type_any_name: INDEX */ +#line 37 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_INDEX; } +#line 29642 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1328: /* comment_on_type_any_name: SCHEMA */ +#line 38 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_SCHEMA; } +#line 29648 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1329: /* comment_on_type_any_name: TYPE_P */ +#line 39 "third_party/libpg_query/grammar/statements/comment_on.y" + { (yyval.objtype) = PG_OBJECT_TYPE; } +#line 29654 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1295: + case 1330: /* ExportStmt: EXPORT_P DATABASE Sconst copy_options */ #line 8 "third_party/libpg_query/grammar/statements/export.y" - { + { PGExportStmt *n = makeNode(PGExportStmt); n->database = NULL; - n->filename = (yyvsp[(3) - (4)].str); + n->filename = (yyvsp[-1].str); n->options = NIL; - if ((yyvsp[(4) - (4)].list)) { - n->options = list_concat(n->options, (yyvsp[(4) - (4)].list)); + if ((yyvsp[0].list)) { + n->options = list_concat(n->options, (yyvsp[0].list)); } (yyval.node) = (PGNode *)n; - ;} + } +#line 29669 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1296: + case 1331: /* ExportStmt: EXPORT_P DATABASE ColId TO Sconst copy_options */ #line 20 "third_party/libpg_query/grammar/statements/export.y" - { + { PGExportStmt *n = makeNode(PGExportStmt); - n->database = (yyvsp[(3) - (6)].str); - n->filename = (yyvsp[(5) - (6)].str); + n->database = (yyvsp[-3].str); + n->filename = (yyvsp[-1].str); n->options = NIL; - if ((yyvsp[(6) - (6)].list)) { - n->options = list_concat(n->options, (yyvsp[(6) - (6)].list)); + if ((yyvsp[0].list)) { + n->options = list_concat(n->options, (yyvsp[0].list)); } (yyval.node) = (PGNode *)n; - ;} + } +#line 29684 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1297: + case 1332: /* ImportStmt: IMPORT_P DATABASE Sconst */ #line 34 "third_party/libpg_query/grammar/statements/export.y" - { + { PGImportStmt *n = makeNode(PGImportStmt); - n->filename = (yyvsp[(3) - (3)].str); + n->filename = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 29694 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1298: + case 1333: /* ExplainStmt: EXPLAIN ExplainableStmt */ #line 10 "third_party/libpg_query/grammar/statements/explain.y" - { + { PGExplainStmt *n = makeNode(PGExplainStmt); - n->query = (yyvsp[(2) - (2)].node); + n->query = (yyvsp[0].node); n->options = NIL; (yyval.node) = (PGNode *) n; - ;} + } +#line 29705 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1299: + case 1334: /* ExplainStmt: EXPLAIN analyze_keyword opt_verbose ExplainableStmt */ #line 17 "third_party/libpg_query/grammar/statements/explain.y" - { + { PGExplainStmt *n = makeNode(PGExplainStmt); - n->query = (yyvsp[(4) - (4)].node); - n->options = list_make1(makeDefElem("analyze", NULL, (yylsp[(2) - (4)]))); - if ((yyvsp[(3) - (4)].boolean)) + n->query = (yyvsp[0].node); + n->options = list_make1(makeDefElem("analyze", NULL, (yylsp[-2]))); + if ((yyvsp[-1].boolean)) n->options = lappend(n->options, - makeDefElem("verbose", NULL, (yylsp[(3) - (4)]))); + makeDefElem("verbose", NULL, (yylsp[-1]))); (yyval.node) = (PGNode *) n; - ;} + } +#line 29719 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1300: + case 1335: /* ExplainStmt: EXPLAIN VERBOSE ExplainableStmt */ #line 27 "third_party/libpg_query/grammar/statements/explain.y" - { + { PGExplainStmt *n = makeNode(PGExplainStmt); - n->query = (yyvsp[(3) - (3)].node); - n->options = list_make1(makeDefElem("verbose", NULL, (yylsp[(2) - (3)]))); + n->query = (yyvsp[0].node); + n->options = list_make1(makeDefElem("verbose", NULL, (yylsp[-1]))); (yyval.node) = (PGNode *) n; - ;} + } +#line 29730 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1301: + case 1336: /* ExplainStmt: EXPLAIN '(' explain_option_list ')' ExplainableStmt */ #line 34 "third_party/libpg_query/grammar/statements/explain.y" - { + { PGExplainStmt *n = makeNode(PGExplainStmt); - n->query = (yyvsp[(5) - (5)].node); - n->options = (yyvsp[(3) - (5)].list); + n->query = (yyvsp[0].node); + n->options = (yyvsp[-2].list); (yyval.node) = (PGNode *) n; - ;} + } +#line 29741 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1302: + case 1337: /* opt_verbose: VERBOSE */ #line 44 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 29747 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1303: + case 1338: /* opt_verbose: %empty */ #line 45 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 29753 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1304: + case 1339: /* explain_option_arg: opt_boolean_or_string */ #line 50 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.node) = (PGNode *) makeString((yyvsp[(1) - (1)].str)); ;} + { (yyval.node) = (PGNode *) makeString((yyvsp[0].str)); } +#line 29759 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1305: + case 1340: /* explain_option_arg: NumericOnly */ #line 51 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.node) = (PGNode *) (yyvsp[(1) - (1)].value); ;} + { (yyval.node) = (PGNode *) (yyvsp[0].value); } +#line 29765 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1306: + case 1341: /* explain_option_arg: %empty */ #line 52 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 29771 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1337: + case 1372: /* NonReservedWord: IDENT */ #line 90 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29777 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1338: + case 1373: /* NonReservedWord: unreserved_keyword */ #line 91 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29783 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1339: + case 1374: /* NonReservedWord: other_keyword */ #line 92 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = pstrdup((yyvsp[(1) - (1)].keyword)); ;} + { (yyval.str) = pstrdup((yyvsp[0].keyword)); } +#line 29789 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1340: + case 1375: /* NonReservedWord_or_Sconst: NonReservedWord */ #line 97 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29795 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1341: + case 1376: /* NonReservedWord_or_Sconst: Sconst */ #line 98 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29801 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1342: + case 1377: /* explain_option_list: explain_option_elem */ #line 104 "third_party/libpg_query/grammar/statements/explain.y" - { - (yyval.list) = list_make1((yyvsp[(1) - (1)].defelt)); - ;} + { + (yyval.list) = list_make1((yyvsp[0].defelt)); + } +#line 29809 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1343: + case 1378: /* explain_option_list: explain_option_list ',' explain_option_elem */ #line 108 "third_party/libpg_query/grammar/statements/explain.y" - { - (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].defelt)); - ;} + { + (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].defelt)); + } +#line 29817 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1344: + case 1379: /* analyze_keyword: ANALYZE */ #line 115 "third_party/libpg_query/grammar/statements/explain.y" - {;} + {} +#line 29823 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1345: + case 1380: /* analyze_keyword: ANALYSE */ #line 116 "third_party/libpg_query/grammar/statements/explain.y" - {;} + {} +#line 29829 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1346: + case 1381: /* opt_boolean_or_string: TRUE_P */ #line 121 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (char*) "true"; ;} + { (yyval.str) = (char*) "true"; } +#line 29835 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1347: + case 1382: /* opt_boolean_or_string: FALSE_P */ #line 122 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (char*) "false"; ;} + { (yyval.str) = (char*) "false"; } +#line 29841 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1348: + case 1383: /* opt_boolean_or_string: ON */ #line 123 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (char*) "on"; ;} + { (yyval.str) = (char*) "on"; } +#line 29847 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1349: + case 1384: /* opt_boolean_or_string: NonReservedWord_or_Sconst */ #line 129 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29853 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1350: + case 1385: /* explain_option_elem: explain_option_name explain_option_arg */ #line 135 "third_party/libpg_query/grammar/statements/explain.y" - { - (yyval.defelt) = makeDefElem((yyvsp[(1) - (2)].str), (yyvsp[(2) - (2)].node), (yylsp[(1) - (2)])); - ;} + { + (yyval.defelt) = makeDefElem((yyvsp[-1].str), (yyvsp[0].node), (yylsp[-1])); + } +#line 29861 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1351: + case 1386: /* explain_option_name: NonReservedWord */ #line 142 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 29867 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1352: + case 1387: /* explain_option_name: analyze_keyword */ #line 143 "third_party/libpg_query/grammar/statements/explain.y" - { (yyval.str) = (char*) "analyze"; ;} + { (yyval.str) = (char*) "analyze"; } +#line 29873 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1353: + case 1388: /* VariableSetStmt: SET set_rest */ #line 11 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGVariableSetStmt *n = (yyvsp[(2) - (2)].vsetstmt); + { + PGVariableSetStmt *n = (yyvsp[0].vsetstmt); n->scope = VAR_SET_SCOPE_DEFAULT; (yyval.node) = (PGNode *) n; - ;} + } +#line 29883 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1354: + case 1389: /* VariableSetStmt: SET LOCAL set_rest */ #line 17 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); + { + PGVariableSetStmt *n = (yyvsp[0].vsetstmt); n->scope = VAR_SET_SCOPE_LOCAL; (yyval.node) = (PGNode *) n; - ;} + } +#line 29893 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1355: + case 1390: /* VariableSetStmt: SET SESSION set_rest */ #line 23 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); + { + PGVariableSetStmt *n = (yyvsp[0].vsetstmt); n->scope = VAR_SET_SCOPE_SESSION; (yyval.node) = (PGNode *) n; - ;} + } +#line 29903 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1356: + case 1391: /* VariableSetStmt: SET GLOBAL set_rest */ #line 29 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGVariableSetStmt *n = (yyvsp[(3) - (3)].vsetstmt); + { + PGVariableSetStmt *n = (yyvsp[0].vsetstmt); n->scope = VAR_SET_SCOPE_GLOBAL; (yyval.node) = (PGNode *) n; - ;} + } +#line 29913 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1357: + case 1392: /* set_rest: generic_set */ #line 38 "third_party/libpg_query/grammar/statements/variable_set.y" - {(yyval.vsetstmt) = (yyvsp[(1) - (1)].vsetstmt);;} + {(yyval.vsetstmt) = (yyvsp[0].vsetstmt);} +#line 29919 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1358: + case 1393: /* set_rest: var_name FROM CURRENT_P */ #line 40 "third_party/libpg_query/grammar/statements/variable_set.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_SET_CURRENT; - n->name = (yyvsp[(1) - (3)].str); + n->name = (yyvsp[-2].str); (yyval.vsetstmt) = n; - ;} + } +#line 29930 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1359: + case 1394: /* set_rest: TIME ZONE zone_value */ #line 48 "third_party/libpg_query/grammar/statements/variable_set.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_SET_VALUE; n->name = (char*) "timezone"; - if ((yyvsp[(3) - (3)].node) != NULL) - n->args = list_make1((yyvsp[(3) - (3)].node)); + if ((yyvsp[0].node) != NULL) + n->args = list_make1((yyvsp[0].node)); else n->kind = VAR_SET_DEFAULT; (yyval.vsetstmt) = n; - ;} + } +#line 29945 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1360: + case 1395: /* set_rest: SCHEMA Sconst */ #line 59 "third_party/libpg_query/grammar/statements/variable_set.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_SET_VALUE; n->name = (char*) "search_path"; - n->args = list_make1(makeStringConst((yyvsp[(2) - (2)].str), (yylsp[(2) - (2)]))); + n->args = list_make1(makeStringConst((yyvsp[0].str), (yylsp[0]))); (yyval.vsetstmt) = n; - ;} + } +#line 29957 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1361: + case 1396: /* generic_set: var_name TO var_list */ #line 71 "third_party/libpg_query/grammar/statements/variable_set.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_SET_VALUE; - n->name = (yyvsp[(1) - (3)].str); - n->args = (yyvsp[(3) - (3)].list); + n->name = (yyvsp[-2].str); + n->args = (yyvsp[0].list); (yyval.vsetstmt) = n; - ;} + } +#line 29969 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1362: + case 1397: /* generic_set: var_name '=' var_list */ #line 79 "third_party/libpg_query/grammar/statements/variable_set.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_SET_VALUE; - n->name = (yyvsp[(1) - (3)].str); - n->args = (yyvsp[(3) - (3)].list); + n->name = (yyvsp[-2].str); + n->args = (yyvsp[0].list); (yyval.vsetstmt) = n; - ;} + } +#line 29981 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1363: + case 1398: /* var_value: a_expr */ #line 90 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.node) = (yyvsp[(1) - (1)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 29987 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1364: + case 1399: /* zone_value: Sconst */ #line 96 "third_party/libpg_query/grammar/statements/variable_set.y" - { - (yyval.node) = makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + { + (yyval.node) = makeStringConst((yyvsp[0].str), (yylsp[0])); + } +#line 29995 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1365: + case 1400: /* zone_value: IDENT */ #line 100 "third_party/libpg_query/grammar/statements/variable_set.y" - { - (yyval.node) = makeStringConst((yyvsp[(1) - (1)].str), (yylsp[(1) - (1)])); - ;} + { + (yyval.node) = makeStringConst((yyvsp[0].str), (yylsp[0])); + } +#line 30003 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1366: + case 1401: /* zone_value: ConstInterval Sconst opt_interval */ #line 104 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGTypeName *t = (yyvsp[(1) - (3)].typnam); - if ((yyvsp[(3) - (3)].list) != NIL) + { + PGTypeName *t = (yyvsp[-2].typnam); + if ((yyvsp[0].list) != NIL) { - PGAConst *n = (PGAConst *) linitial((yyvsp[(3) - (3)].list)); + PGAConst *n = (PGAConst *) linitial((yyvsp[0].list)); if ((n->val.val.ival & ~(INTERVAL_MASK(HOUR) | INTERVAL_MASK(MINUTE))) != 0) ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), errmsg("time zone interval must be HOUR or HOUR TO MINUTE"), - parser_errposition((yylsp[(3) - (3)])))); + parser_errposition((yylsp[0])))); } - t->typmods = (yyvsp[(3) - (3)].list); - (yyval.node) = makeStringConstCast((yyvsp[(2) - (3)].str), (yylsp[(2) - (3)]), t); - ;} + t->typmods = (yyvsp[0].list); + (yyval.node) = makeStringConstCast((yyvsp[-1].str), (yylsp[-1]), t); + } +#line 30022 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1367: + case 1402: /* zone_value: ConstInterval '(' Iconst ')' Sconst */ #line 119 "third_party/libpg_query/grammar/statements/variable_set.y" - { - PGTypeName *t = (yyvsp[(1) - (5)].typnam); + { + PGTypeName *t = (yyvsp[-4].typnam); t->typmods = list_make2(makeIntConst(INTERVAL_FULL_RANGE, -1), - makeIntConst((yyvsp[(3) - (5)].ival), (yylsp[(3) - (5)]))); - (yyval.node) = makeStringConstCast((yyvsp[(5) - (5)].str), (yylsp[(5) - (5)]), t); - ;} + makeIntConst((yyvsp[-2].ival), (yylsp[-2]))); + (yyval.node) = makeStringConstCast((yyvsp[0].str), (yylsp[0]), t); + } +#line 30033 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1368: + case 1403: /* zone_value: NumericOnly */ #line 125 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.node) = makeAConst((yyvsp[(1) - (1)].value), (yylsp[(1) - (1)])); ;} + { (yyval.node) = makeAConst((yyvsp[0].value), (yylsp[0])); } +#line 30039 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1369: + case 1404: /* zone_value: DEFAULT */ #line 126 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 30045 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1370: + case 1405: /* zone_value: LOCAL */ #line 127 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 30051 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1371: + case 1406: /* var_list: var_value */ #line 131 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.list) = list_make1((yyvsp[(1) - (1)].node)); ;} + { (yyval.list) = list_make1((yyvsp[0].node)); } +#line 30057 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1372: + case 1407: /* var_list: var_list ',' var_value */ #line 132 "third_party/libpg_query/grammar/statements/variable_set.y" - { (yyval.list) = lappend((yyvsp[(1) - (3)].list), (yyvsp[(3) - (3)].node)); ;} + { (yyval.list) = lappend((yyvsp[-2].list), (yyvsp[0].node)); } +#line 30063 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1373: + case 1408: /* LoadStmt: LOAD file_name */ #line 8 "third_party/libpg_query/grammar/statements/load.y" - { + { PGLoadStmt *n = makeNode(PGLoadStmt); - n->filename = (yyvsp[(2) - (2)].str); + n->filename = (yyvsp[0].str); n->repository = ""; n->load_type = PG_LOAD_TYPE_LOAD; (yyval.node) = (PGNode *)n; - ;} + } +#line 30075 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1374: + case 1409: /* LoadStmt: INSTALL file_name */ #line 15 "third_party/libpg_query/grammar/statements/load.y" - { + { PGLoadStmt *n = makeNode(PGLoadStmt); - n->filename = (yyvsp[(2) - (2)].str); + n->filename = (yyvsp[0].str); n->repository = ""; n->load_type = PG_LOAD_TYPE_INSTALL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30087 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1375: + case 1410: /* LoadStmt: FORCE INSTALL file_name */ #line 22 "third_party/libpg_query/grammar/statements/load.y" - { + { PGLoadStmt *n = makeNode(PGLoadStmt); - n->filename = (yyvsp[(3) - (3)].str); + n->filename = (yyvsp[0].str); n->repository = ""; n->load_type = PG_LOAD_TYPE_FORCE_INSTALL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30099 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1376: + case 1411: /* LoadStmt: INSTALL file_name FROM repo_path */ #line 29 "third_party/libpg_query/grammar/statements/load.y" - { + { PGLoadStmt *n = makeNode(PGLoadStmt); - n->filename = (yyvsp[(2) - (4)].str); - n->repository = (yyvsp[(4) - (4)].str); + n->filename = (yyvsp[-2].str); + n->repository = (yyvsp[0].str); n->load_type = PG_LOAD_TYPE_INSTALL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30111 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1377: + case 1412: /* LoadStmt: FORCE INSTALL file_name FROM repo_path */ #line 36 "third_party/libpg_query/grammar/statements/load.y" - { + { PGLoadStmt *n = makeNode(PGLoadStmt); - n->filename = (yyvsp[(3) - (5)].str); - n->repository = (yyvsp[(5) - (5)].str); + n->filename = (yyvsp[-2].str); + n->repository = (yyvsp[0].str); n->load_type = PG_LOAD_TYPE_FORCE_INSTALL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30123 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1378: + case 1413: /* file_name: Sconst */ #line 45 "third_party/libpg_query/grammar/statements/load.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 30129 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1379: + case 1414: /* file_name: ColId */ #line 46 "third_party/libpg_query/grammar/statements/load.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 30135 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1380: + case 1415: /* repo_path: Sconst */ #line 48 "third_party/libpg_query/grammar/statements/load.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 30141 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1381: + case 1416: /* repo_path: ColId */ #line 49 "third_party/libpg_query/grammar/statements/load.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 30147 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1382: + case 1417: /* VacuumStmt: VACUUM opt_full opt_freeze opt_verbose */ #line 9 "third_party/libpg_query/grammar/statements/vacuum.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); n->options = PG_VACOPT_VACUUM; - if ((yyvsp[(2) - (4)].boolean)) + if ((yyvsp[-2].boolean)) n->options |= PG_VACOPT_FULL; - if ((yyvsp[(3) - (4)].boolean)) + if ((yyvsp[-1].boolean)) n->options |= PG_VACOPT_FREEZE; - if ((yyvsp[(4) - (4)].boolean)) + if ((yyvsp[0].boolean)) n->options |= PG_VACOPT_VERBOSE; n->relation = NULL; n->va_cols = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30165 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1383: + case 1418: /* VacuumStmt: VACUUM opt_full opt_freeze opt_verbose qualified_name opt_name_list */ #line 23 "third_party/libpg_query/grammar/statements/vacuum.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); n->options = PG_VACOPT_VACUUM; - if ((yyvsp[(2) - (6)].boolean)) + if ((yyvsp[-4].boolean)) n->options |= PG_VACOPT_FULL; - if ((yyvsp[(3) - (6)].boolean)) + if ((yyvsp[-3].boolean)) n->options |= PG_VACOPT_FREEZE; - if ((yyvsp[(4) - (6)].boolean)) + if ((yyvsp[-2].boolean)) n->options |= PG_VACOPT_VERBOSE; - n->relation = (yyvsp[(5) - (6)].range); - n->va_cols = (yyvsp[(6) - (6)].list); + n->relation = (yyvsp[-1].range); + n->va_cols = (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 30183 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1384: + case 1419: /* VacuumStmt: VACUUM opt_full opt_freeze opt_verbose AnalyzeStmt */ #line 37 "third_party/libpg_query/grammar/statements/vacuum.y" - { - PGVacuumStmt *n = (PGVacuumStmt *) (yyvsp[(5) - (5)].node); + { + PGVacuumStmt *n = (PGVacuumStmt *) (yyvsp[0].node); n->options |= PG_VACOPT_VACUUM; - if ((yyvsp[(2) - (5)].boolean)) + if ((yyvsp[-3].boolean)) n->options |= PG_VACOPT_FULL; - if ((yyvsp[(3) - (5)].boolean)) + if ((yyvsp[-2].boolean)) n->options |= PG_VACOPT_FREEZE; - if ((yyvsp[(4) - (5)].boolean)) + if ((yyvsp[-1].boolean)) n->options |= PG_VACOPT_VERBOSE; (yyval.node) = (PGNode *)n; - ;} + } +#line 30199 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1385: + case 1420: /* VacuumStmt: VACUUM '(' vacuum_option_list ')' */ #line 49 "third_party/libpg_query/grammar/statements/vacuum.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); - n->options = PG_VACOPT_VACUUM | (yyvsp[(3) - (4)].ival); + n->options = PG_VACOPT_VACUUM | (yyvsp[-1].ival); n->relation = NULL; n->va_cols = NIL; (yyval.node) = (PGNode *) n; - ;} + } +#line 30211 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1386: + case 1421: /* VacuumStmt: VACUUM '(' vacuum_option_list ')' qualified_name opt_name_list */ #line 57 "third_party/libpg_query/grammar/statements/vacuum.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); - n->options = PG_VACOPT_VACUUM | (yyvsp[(3) - (6)].ival); - n->relation = (yyvsp[(5) - (6)].range); - n->va_cols = (yyvsp[(6) - (6)].list); + n->options = PG_VACOPT_VACUUM | (yyvsp[-3].ival); + n->relation = (yyvsp[-1].range); + n->va_cols = (yyvsp[0].list); if (n->va_cols != NIL) /* implies analyze */ n->options |= PG_VACOPT_ANALYZE; (yyval.node) = (PGNode *) n; - ;} + } +#line 30225 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1387: + case 1422: /* vacuum_option_elem: analyze_keyword */ #line 70 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = PG_VACOPT_ANALYZE; ;} + { (yyval.ival) = PG_VACOPT_ANALYZE; } +#line 30231 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1388: + case 1423: /* vacuum_option_elem: VERBOSE */ #line 71 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = PG_VACOPT_VERBOSE; ;} + { (yyval.ival) = PG_VACOPT_VERBOSE; } +#line 30237 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1389: + case 1424: /* vacuum_option_elem: FREEZE */ #line 72 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = PG_VACOPT_FREEZE; ;} + { (yyval.ival) = PG_VACOPT_FREEZE; } +#line 30243 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1390: + case 1425: /* vacuum_option_elem: FULL */ #line 73 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = PG_VACOPT_FULL; ;} + { (yyval.ival) = PG_VACOPT_FULL; } +#line 30249 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1391: + case 1426: /* vacuum_option_elem: IDENT */ #line 75 "third_party/libpg_query/grammar/statements/vacuum.y" - { - if (strcmp((yyvsp[(1) - (1)].str), "disable_page_skipping") == 0) + { + if (strcmp((yyvsp[0].str), "disable_page_skipping") == 0) (yyval.ival) = PG_VACOPT_DISABLE_PAGE_SKIPPING; else ereport(ERROR, (errcode(PG_ERRCODE_SYNTAX_ERROR), - errmsg("unrecognized VACUUM option \"%s\"", (yyvsp[(1) - (1)].str)), - parser_errposition((yylsp[(1) - (1)])))); - ;} + errmsg("unrecognized VACUUM option \"%s\"", (yyvsp[0].str)), + parser_errposition((yylsp[0])))); + } +#line 30263 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1392: + case 1427: /* opt_full: FULL */ #line 87 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 30269 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1393: + case 1428: /* opt_full: %empty */ #line 88 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 30275 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1394: + case 1429: /* vacuum_option_list: vacuum_option_elem */ #line 93 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = (yyvsp[(1) - (1)].ival); ;} + { (yyval.ival) = (yyvsp[0].ival); } +#line 30281 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1395: + case 1430: /* vacuum_option_list: vacuum_option_list ',' vacuum_option_elem */ #line 94 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.ival) = (yyvsp[(1) - (3)].ival) | (yyvsp[(3) - (3)].ival); ;} + { (yyval.ival) = (yyvsp[-2].ival) | (yyvsp[0].ival); } +#line 30287 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1396: + case 1431: /* opt_freeze: FREEZE */ #line 98 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 30293 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1397: + case 1432: /* opt_freeze: %empty */ #line 99 "third_party/libpg_query/grammar/statements/vacuum.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 30299 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1398: + case 1433: /* DeleteStmt: opt_with_clause DELETE_P FROM relation_expr_opt_alias using_clause where_or_current_clause returning_clause */ #line 9 "third_party/libpg_query/grammar/statements/delete.y" - { + { PGDeleteStmt *n = makeNode(PGDeleteStmt); - n->relation = (yyvsp[(4) - (7)].range); - n->usingClause = (yyvsp[(5) - (7)].list); - n->whereClause = (yyvsp[(6) - (7)].node); - n->returningList = (yyvsp[(7) - (7)].list); - n->withClause = (yyvsp[(1) - (7)].with); + n->relation = (yyvsp[-3].range); + n->usingClause = (yyvsp[-2].list); + n->whereClause = (yyvsp[-1].node); + n->returningList = (yyvsp[0].list); + n->withClause = (yyvsp[-6].with); (yyval.node) = (PGNode *)n; - ;} + } +#line 30313 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1399: + case 1434: /* DeleteStmt: TRUNCATE opt_table relation_expr_opt_alias */ #line 19 "third_party/libpg_query/grammar/statements/delete.y" - { + { PGDeleteStmt *n = makeNode(PGDeleteStmt); - n->relation = (yyvsp[(3) - (3)].range); + n->relation = (yyvsp[0].range); n->usingClause = NULL; n->whereClause = NULL; n->returningList = NULL; n->withClause = NULL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30327 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1400: + case 1435: /* relation_expr_opt_alias: relation_expr */ #line 32 "third_party/libpg_query/grammar/statements/delete.y" - { - (yyval.range) = (yyvsp[(1) - (1)].range); - ;} + { + (yyval.range) = (yyvsp[0].range); + } +#line 30335 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1401: + case 1436: /* relation_expr_opt_alias: relation_expr ColId */ #line 36 "third_party/libpg_query/grammar/statements/delete.y" - { + { PGAlias *alias = makeNode(PGAlias); - alias->aliasname = (yyvsp[(2) - (2)].str); - (yyvsp[(1) - (2)].range)->alias = alias; - (yyval.range) = (yyvsp[(1) - (2)].range); - ;} + alias->aliasname = (yyvsp[0].str); + (yyvsp[-1].range)->alias = alias; + (yyval.range) = (yyvsp[-1].range); + } +#line 30346 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1402: + case 1437: /* relation_expr_opt_alias: relation_expr AS ColId */ #line 43 "third_party/libpg_query/grammar/statements/delete.y" - { + { PGAlias *alias = makeNode(PGAlias); - alias->aliasname = (yyvsp[(3) - (3)].str); - (yyvsp[(1) - (3)].range)->alias = alias; - (yyval.range) = (yyvsp[(1) - (3)].range); - ;} + alias->aliasname = (yyvsp[0].str); + (yyvsp[-2].range)->alias = alias; + (yyval.range) = (yyvsp[-2].range); + } +#line 30357 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1403: + case 1438: /* where_or_current_clause: WHERE a_expr */ #line 53 "third_party/libpg_query/grammar/statements/delete.y" - { (yyval.node) = (yyvsp[(2) - (2)].node); ;} + { (yyval.node) = (yyvsp[0].node); } +#line 30363 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1404: + case 1439: /* where_or_current_clause: %empty */ #line 54 "third_party/libpg_query/grammar/statements/delete.y" - { (yyval.node) = NULL; ;} + { (yyval.node) = NULL; } +#line 30369 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1405: + case 1440: /* using_clause: USING from_list_opt_comma */ #line 60 "third_party/libpg_query/grammar/statements/delete.y" - { (yyval.list) = (yyvsp[(2) - (2)].list); ;} + { (yyval.list) = (yyvsp[0].list); } +#line 30375 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1406: + case 1441: /* using_clause: %empty */ #line 61 "third_party/libpg_query/grammar/statements/delete.y" - { (yyval.list) = NIL; ;} + { (yyval.list) = NIL; } +#line 30381 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1407: + case 1442: /* AnalyzeStmt: analyze_keyword opt_verbose */ #line 10 "third_party/libpg_query/grammar/statements/analyze.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); n->options = PG_VACOPT_ANALYZE; - if ((yyvsp[(2) - (2)].boolean)) + if ((yyvsp[0].boolean)) n->options |= PG_VACOPT_VERBOSE; n->relation = NULL; n->va_cols = NIL; (yyval.node) = (PGNode *)n; - ;} + } +#line 30395 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1408: + case 1443: /* AnalyzeStmt: analyze_keyword opt_verbose qualified_name opt_name_list */ #line 20 "third_party/libpg_query/grammar/statements/analyze.y" - { + { PGVacuumStmt *n = makeNode(PGVacuumStmt); n->options = PG_VACOPT_ANALYZE; - if ((yyvsp[(2) - (4)].boolean)) + if ((yyvsp[-2].boolean)) n->options |= PG_VACOPT_VERBOSE; - n->relation = (yyvsp[(3) - (4)].range); - n->va_cols = (yyvsp[(4) - (4)].list); + n->relation = (yyvsp[-1].range); + n->va_cols = (yyvsp[0].list); (yyval.node) = (PGNode *)n; - ;} + } +#line 30409 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1409: + case 1444: /* AttachStmt: ATTACH opt_database Sconst opt_database_alias copy_options */ #line 8 "third_party/libpg_query/grammar/statements/attach.y" - { + { PGAttachStmt *n = makeNode(PGAttachStmt); - n->path = (yyvsp[(3) - (5)].str); - n->name = (yyvsp[(4) - (5)].str); - n->options = (yyvsp[(5) - (5)].list); + n->path = (yyvsp[-2].str); + n->name = (yyvsp[-1].str); + n->options = (yyvsp[0].list); n->onconflict = PG_ERROR_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 30422 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1410: + case 1445: /* AttachStmt: ATTACH IF_P NOT EXISTS opt_database Sconst opt_database_alias copy_options */ #line 17 "third_party/libpg_query/grammar/statements/attach.y" - { + { PGAttachStmt *n = makeNode(PGAttachStmt); - n->path = (yyvsp[(6) - (8)].str); - n->name = (yyvsp[(7) - (8)].str); - n->options = (yyvsp[(8) - (8)].list); + n->path = (yyvsp[-2].str); + n->name = (yyvsp[-1].str); + n->options = (yyvsp[0].list); n->onconflict = PG_IGNORE_ON_CONFLICT; (yyval.node) = (PGNode *)n; - ;} + } +#line 30435 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1411: + case 1446: /* DetachStmt: DETACH ColLabel */ #line 29 "third_party/libpg_query/grammar/statements/attach.y" - { + { PGDetachStmt *n = makeNode(PGDetachStmt); n->missing_ok = false; - n->db_name = (yyvsp[(3) - (3)].str); + n->db_name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 30446 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1412: + case 1447: /* DetachStmt: DETACH DATABASE ColLabel */ #line 36 "third_party/libpg_query/grammar/statements/attach.y" - { + { + PGDetachStmt *n = makeNode(PGDetachStmt); + n->missing_ok = false; + n->db_name = (yyvsp[0].str); + (yyval.node) = (PGNode *)n; + } +#line 30457 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1448: /* DetachStmt: DETACH DATABASE IF_P EXISTS ColLabel */ +#line 43 "third_party/libpg_query/grammar/statements/attach.y" + { PGDetachStmt *n = makeNode(PGDetachStmt); n->missing_ok = true; - n->db_name = (yyvsp[(5) - (5)].str); + n->db_name = (yyvsp[0].str); (yyval.node) = (PGNode *)n; - ;} + } +#line 30468 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1413: -#line 44 "third_party/libpg_query/grammar/statements/attach.y" - {;} + case 1449: /* opt_database: DATABASE */ +#line 51 "third_party/libpg_query/grammar/statements/attach.y" + {} +#line 30474 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1414: -#line 45 "third_party/libpg_query/grammar/statements/attach.y" - {;} + case 1450: /* opt_database: %empty */ +#line 52 "third_party/libpg_query/grammar/statements/attach.y" + {} +#line 30480 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1415: -#line 49 "third_party/libpg_query/grammar/statements/attach.y" - { (yyval.str) = (yyvsp[(2) - (2)].str); ;} + case 1451: /* opt_database_alias: AS ColId */ +#line 56 "third_party/libpg_query/grammar/statements/attach.y" + { (yyval.str) = (yyvsp[0].str); } +#line 30486 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1416: -#line 50 "third_party/libpg_query/grammar/statements/attach.y" - { (yyval.str) = NULL; ;} + case 1452: /* opt_database_alias: %empty */ +#line 57 "third_party/libpg_query/grammar/statements/attach.y" + { (yyval.str) = NULL; } +#line 30492 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1417: + case 1453: /* VariableResetStmt: RESET reset_rest */ #line 3 "third_party/libpg_query/grammar/statements/variable_reset.y" - { - (yyvsp[(2) - (2)].vsetstmt)->scope = VAR_SET_SCOPE_DEFAULT; - (yyval.node) = (PGNode *) (yyvsp[(2) - (2)].vsetstmt); - ;} + { + (yyvsp[0].vsetstmt)->scope = VAR_SET_SCOPE_DEFAULT; + (yyval.node) = (PGNode *) (yyvsp[0].vsetstmt); + } +#line 30501 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1418: + case 1454: /* VariableResetStmt: RESET LOCAL reset_rest */ #line 8 "third_party/libpg_query/grammar/statements/variable_reset.y" - { - (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_LOCAL; - (yyval.node) = (PGNode *) (yyvsp[(3) - (3)].vsetstmt); - ;} + { + (yyvsp[0].vsetstmt)->scope = VAR_SET_SCOPE_LOCAL; + (yyval.node) = (PGNode *) (yyvsp[0].vsetstmt); + } +#line 30510 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1419: + case 1455: /* VariableResetStmt: RESET SESSION reset_rest */ #line 13 "third_party/libpg_query/grammar/statements/variable_reset.y" - { - (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_SESSION; - (yyval.node) = (PGNode *) (yyvsp[(3) - (3)].vsetstmt); - ;} + { + (yyvsp[0].vsetstmt)->scope = VAR_SET_SCOPE_SESSION; + (yyval.node) = (PGNode *) (yyvsp[0].vsetstmt); + } +#line 30519 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1420: + case 1456: /* VariableResetStmt: RESET GLOBAL reset_rest */ #line 18 "third_party/libpg_query/grammar/statements/variable_reset.y" - { - (yyvsp[(3) - (3)].vsetstmt)->scope = VAR_SET_SCOPE_GLOBAL; - (yyval.node) = (PGNode *) (yyvsp[(3) - (3)].vsetstmt); - ;} + { + (yyvsp[0].vsetstmt)->scope = VAR_SET_SCOPE_GLOBAL; + (yyval.node) = (PGNode *) (yyvsp[0].vsetstmt); + } +#line 30528 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1421: + case 1457: /* generic_reset: var_name */ #line 27 "third_party/libpg_query/grammar/statements/variable_reset.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_RESET; - n->name = (yyvsp[(1) - (1)].str); + n->name = (yyvsp[0].str); (yyval.vsetstmt) = n; - ;} + } +#line 30539 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1422: + case 1458: /* generic_reset: ALL */ #line 34 "third_party/libpg_query/grammar/statements/variable_reset.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_RESET_ALL; (yyval.vsetstmt) = n; - ;} + } +#line 30549 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1423: + case 1459: /* reset_rest: generic_reset */ #line 43 "third_party/libpg_query/grammar/statements/variable_reset.y" - { (yyval.vsetstmt) = (yyvsp[(1) - (1)].vsetstmt); ;} + { (yyval.vsetstmt) = (yyvsp[0].vsetstmt); } +#line 30555 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1424: + case 1460: /* reset_rest: TIME ZONE */ #line 45 "third_party/libpg_query/grammar/statements/variable_reset.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_RESET; n->name = (char*) "timezone"; (yyval.vsetstmt) = n; - ;} + } +#line 30566 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1425: + case 1461: /* reset_rest: TRANSACTION ISOLATION LEVEL */ #line 52 "third_party/libpg_query/grammar/statements/variable_reset.y" - { + { PGVariableSetStmt *n = makeNode(PGVariableSetStmt); n->kind = VAR_RESET; n->name = (char*) "transaction_isolation"; (yyval.vsetstmt) = n; - ;} + } +#line 30577 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1426: + case 1462: /* VariableShowStmt: show_or_describe SelectStmt */ #line 3 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowSelectStmt *n = makeNode(PGVariableShowSelectStmt); - n->stmt = (yyvsp[(2) - (2)].node); + n->stmt = (yyvsp[0].node); n->name = (char*) "select"; n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} + } +#line 30589 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1427: + case 1463: /* VariableShowStmt: SUMMARIZE SelectStmt */ #line 10 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowSelectStmt *n = makeNode(PGVariableShowSelectStmt); - n->stmt = (yyvsp[(2) - (2)].node); + n->stmt = (yyvsp[0].node); n->name = (char*) "select"; n->is_summary = 1; (yyval.node) = (PGNode *) n; - ;} + } +#line 30601 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1428: + case 1464: /* VariableShowStmt: SUMMARIZE table_id */ #line 18 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); n->is_summary = 1; (yyval.node) = (PGNode *) n; - ;} + } +#line 30612 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1429: + case 1465: /* VariableShowStmt: show_or_describe table_id */ #line 25 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); - n->name = (yyvsp[(2) - (2)].str); + n->name = (yyvsp[0].str); n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} + } +#line 30623 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1430: + case 1466: /* VariableShowStmt: show_or_describe TIME ZONE */ #line 32 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); n->name = (char*) "timezone"; n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} + } +#line 30634 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1431: + case 1467: /* VariableShowStmt: show_or_describe TRANSACTION ISOLATION LEVEL */ #line 39 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); n->name = (char*) "transaction_isolation"; n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} + } +#line 30645 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1432: + case 1468: /* VariableShowStmt: show_or_describe ALL opt_tables */ #line 46 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); n->name = (char*) "__show_tables_expanded"; n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} + } +#line 30656 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1433: + case 1469: /* VariableShowStmt: show_or_describe */ #line 53 "third_party/libpg_query/grammar/statements/variable_show.y" - { + { PGVariableShowStmt *n = makeNode(PGVariableShowStmt); n->name = (char*) "__show_tables_expanded"; n->is_summary = 0; (yyval.node) = (PGNode *) n; - ;} - break; - - case 1438: -#line 65 "third_party/libpg_query/grammar/statements/variable_show.y" - { (yyval.str) = (yyvsp[(1) - (1)].str); ;} + } +#line 30667 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1439: + case 1476: /* var_name: ColId */ #line 67 "third_party/libpg_query/grammar/statements/variable_show.y" - { (yyval.str) = psprintf("%s.%s", (yyvsp[(1) - (3)].str), (yyvsp[(3) - (3)].str)); ;} + { (yyval.str) = (yyvsp[0].str); } +#line 30673 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1440: -#line 70 "third_party/libpg_query/grammar/statements/variable_show.y" - { (yyval.str) = psprintf("\"%s\"", (yyvsp[(1) - (1)].str)); ;} + case 1477: /* var_name: var_name '.' ColId */ +#line 69 "third_party/libpg_query/grammar/statements/variable_show.y" + { (yyval.str) = psprintf("%s.%s", (yyvsp[-2].str), (yyvsp[0].str)); } +#line 30679 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1441: + case 1478: /* table_id: ColId */ #line 72 "third_party/libpg_query/grammar/statements/variable_show.y" - { (yyval.str) = psprintf("%s.\"%s\"", (yyvsp[(1) - (3)].str), (yyvsp[(3) - (3)].str)); ;} + { (yyval.str) = psprintf("\"%s\"", (yyvsp[0].str)); } +#line 30685 "third_party/libpg_query/grammar/grammar_out.cpp" + break; + + case 1479: /* table_id: table_id '.' ColId */ +#line 74 "third_party/libpg_query/grammar/statements/variable_show.y" + { (yyval.str) = psprintf("%s.\"%s\"", (yyvsp[-2].str), (yyvsp[0].str)); } +#line 30691 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1442: + case 1480: /* CallStmt: CALL_P func_application */ #line 7 "third_party/libpg_query/grammar/statements/call.y" - { + { PGCallStmt *n = makeNode(PGCallStmt); - n->func = (yyvsp[(2) - (2)].node); + n->func = (yyvsp[0].node); (yyval.node) = (PGNode *) n; - ;} + } +#line 30701 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1443: + case 1481: /* ViewStmt: CREATE_P OptTemp VIEW qualified_name opt_column_list opt_reloptions AS SelectStmt opt_check_option */ #line 10 "third_party/libpg_query/grammar/statements/view.y" - { + { PGViewStmt *n = makeNode(PGViewStmt); - n->view = (yyvsp[(4) - (9)].range); - n->view->relpersistence = (yyvsp[(2) - (9)].ival); - n->aliases = (yyvsp[(5) - (9)].list); - n->query = (yyvsp[(8) - (9)].node); + n->view = (yyvsp[-5].range); + n->view->relpersistence = (yyvsp[-7].ival); + n->aliases = (yyvsp[-4].list); + n->query = (yyvsp[-1].node); n->onconflict = PG_ERROR_ON_CONFLICT; - n->options = (yyvsp[(6) - (9)].list); - n->withCheckOption = (yyvsp[(9) - (9)].viewcheckoption); + n->options = (yyvsp[-3].list); + n->withCheckOption = (yyvsp[0].viewcheckoption); (yyval.node) = (PGNode *) n; - ;} + } +#line 30717 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1444: + case 1482: /* ViewStmt: CREATE_P OptTemp VIEW IF_P NOT EXISTS qualified_name opt_column_list opt_reloptions AS SelectStmt opt_check_option */ #line 23 "third_party/libpg_query/grammar/statements/view.y" - { + { PGViewStmt *n = makeNode(PGViewStmt); - n->view = (yyvsp[(7) - (12)].range); - n->view->relpersistence = (yyvsp[(2) - (12)].ival); - n->aliases = (yyvsp[(8) - (12)].list); - n->query = (yyvsp[(11) - (12)].node); + n->view = (yyvsp[-5].range); + n->view->relpersistence = (yyvsp[-10].ival); + n->aliases = (yyvsp[-4].list); + n->query = (yyvsp[-1].node); n->onconflict = PG_IGNORE_ON_CONFLICT; - n->options = (yyvsp[(9) - (12)].list); - n->withCheckOption = (yyvsp[(12) - (12)].viewcheckoption); + n->options = (yyvsp[-3].list); + n->withCheckOption = (yyvsp[0].viewcheckoption); (yyval.node) = (PGNode *) n; - ;} + } +#line 30733 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1445: + case 1483: /* ViewStmt: CREATE_P OR REPLACE OptTemp VIEW qualified_name opt_column_list opt_reloptions AS SelectStmt opt_check_option */ #line 36 "third_party/libpg_query/grammar/statements/view.y" - { + { PGViewStmt *n = makeNode(PGViewStmt); - n->view = (yyvsp[(6) - (11)].range); - n->view->relpersistence = (yyvsp[(4) - (11)].ival); - n->aliases = (yyvsp[(7) - (11)].list); - n->query = (yyvsp[(10) - (11)].node); + n->view = (yyvsp[-5].range); + n->view->relpersistence = (yyvsp[-7].ival); + n->aliases = (yyvsp[-4].list); + n->query = (yyvsp[-1].node); n->onconflict = PG_REPLACE_ON_CONFLICT; - n->options = (yyvsp[(8) - (11)].list); - n->withCheckOption = (yyvsp[(11) - (11)].viewcheckoption); + n->options = (yyvsp[-3].list); + n->withCheckOption = (yyvsp[0].viewcheckoption); (yyval.node) = (PGNode *) n; - ;} + } +#line 30749 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1446: + case 1484: /* ViewStmt: CREATE_P OptTemp RECURSIVE VIEW qualified_name '(' columnList ')' opt_reloptions AS SelectStmt opt_check_option */ #line 49 "third_party/libpg_query/grammar/statements/view.y" - { + { PGViewStmt *n = makeNode(PGViewStmt); - n->view = (yyvsp[(5) - (12)].range); - n->view->relpersistence = (yyvsp[(2) - (12)].ival); - n->aliases = (yyvsp[(7) - (12)].list); - n->query = makeRecursiveViewSelect(n->view->relname, n->aliases, (yyvsp[(11) - (12)].node)); + n->view = (yyvsp[-7].range); + n->view->relpersistence = (yyvsp[-10].ival); + n->aliases = (yyvsp[-5].list); + n->query = makeRecursiveViewSelect(n->view->relname, n->aliases, (yyvsp[-1].node)); n->onconflict = PG_ERROR_ON_CONFLICT; - n->options = (yyvsp[(9) - (12)].list); - n->withCheckOption = (yyvsp[(12) - (12)].viewcheckoption); + n->options = (yyvsp[-3].list); + n->withCheckOption = (yyvsp[0].viewcheckoption); if (n->withCheckOption != PG_NO_CHECK_OPTION) ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("WITH CHECK OPTION not supported on recursive views"), - parser_errposition((yylsp[(12) - (12)])))); + parser_errposition((yylsp[0])))); (yyval.node) = (PGNode *) n; - ;} + } +#line 30770 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1447: + case 1485: /* ViewStmt: CREATE_P OR REPLACE OptTemp RECURSIVE VIEW qualified_name '(' columnList ')' opt_reloptions AS SelectStmt opt_check_option */ #line 67 "third_party/libpg_query/grammar/statements/view.y" - { + { PGViewStmt *n = makeNode(PGViewStmt); - n->view = (yyvsp[(7) - (14)].range); - n->view->relpersistence = (yyvsp[(4) - (14)].ival); - n->aliases = (yyvsp[(9) - (14)].list); - n->query = makeRecursiveViewSelect(n->view->relname, n->aliases, (yyvsp[(13) - (14)].node)); + n->view = (yyvsp[-7].range); + n->view->relpersistence = (yyvsp[-10].ival); + n->aliases = (yyvsp[-5].list); + n->query = makeRecursiveViewSelect(n->view->relname, n->aliases, (yyvsp[-1].node)); n->onconflict = PG_REPLACE_ON_CONFLICT; - n->options = (yyvsp[(11) - (14)].list); - n->withCheckOption = (yyvsp[(14) - (14)].viewcheckoption); + n->options = (yyvsp[-3].list); + n->withCheckOption = (yyvsp[0].viewcheckoption); if (n->withCheckOption != PG_NO_CHECK_OPTION) ereport(ERROR, (errcode(PG_ERRCODE_FEATURE_NOT_SUPPORTED), errmsg("WITH CHECK OPTION not supported on recursive views"), - parser_errposition((yylsp[(14) - (14)])))); + parser_errposition((yylsp[0])))); (yyval.node) = (PGNode *) n; - ;} + } +#line 30791 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1448: + case 1486: /* opt_check_option: WITH CHECK_P OPTION */ #line 87 "third_party/libpg_query/grammar/statements/view.y" - { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; ;} + { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; } +#line 30797 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1449: + case 1487: /* opt_check_option: WITH CASCADED CHECK_P OPTION */ #line 88 "third_party/libpg_query/grammar/statements/view.y" - { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; ;} + { (yyval.viewcheckoption) = CASCADED_CHECK_OPTION; } +#line 30803 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1450: + case 1488: /* opt_check_option: WITH LOCAL CHECK_P OPTION */ #line 89 "third_party/libpg_query/grammar/statements/view.y" - { (yyval.viewcheckoption) = PG_LOCAL_CHECK_OPTION; ;} + { (yyval.viewcheckoption) = PG_LOCAL_CHECK_OPTION; } +#line 30809 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1451: + case 1489: /* opt_check_option: %empty */ #line 90 "third_party/libpg_query/grammar/statements/view.y" - { (yyval.viewcheckoption) = PG_NO_CHECK_OPTION; ;} + { (yyval.viewcheckoption) = PG_NO_CHECK_OPTION; } +#line 30815 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1452: + case 1490: /* CreateAsStmt: CREATE_P OptTemp TABLE create_as_target AS SelectStmt opt_with_data */ #line 12 "third_party/libpg_query/grammar/statements/create_as.y" - { + { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); - ctas->query = (yyvsp[(6) - (7)].node); - ctas->into = (yyvsp[(4) - (7)].into); + ctas->query = (yyvsp[-1].node); + ctas->into = (yyvsp[-3].into); ctas->relkind = PG_OBJECT_TABLE; ctas->is_select_into = false; ctas->onconflict = PG_ERROR_ON_CONFLICT; /* cram additional flags into the PGIntoClause */ - (yyvsp[(4) - (7)].into)->rel->relpersistence = (yyvsp[(2) - (7)].ival); - (yyvsp[(4) - (7)].into)->skipData = !((yyvsp[(7) - (7)].boolean)); + (yyvsp[-3].into)->rel->relpersistence = (yyvsp[-5].ival); + (yyvsp[-3].into)->skipData = !((yyvsp[0].boolean)); (yyval.node) = (PGNode *) ctas; - ;} + } +#line 30832 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1453: + case 1491: /* CreateAsStmt: CREATE_P OptTemp TABLE IF_P NOT EXISTS create_as_target AS SelectStmt opt_with_data */ #line 25 "third_party/libpg_query/grammar/statements/create_as.y" - { + { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); - ctas->query = (yyvsp[(9) - (10)].node); - ctas->into = (yyvsp[(7) - (10)].into); + ctas->query = (yyvsp[-1].node); + ctas->into = (yyvsp[-3].into); ctas->relkind = PG_OBJECT_TABLE; ctas->is_select_into = false; ctas->onconflict = PG_IGNORE_ON_CONFLICT; /* cram additional flags into the PGIntoClause */ - (yyvsp[(7) - (10)].into)->rel->relpersistence = (yyvsp[(2) - (10)].ival); - (yyvsp[(7) - (10)].into)->skipData = !((yyvsp[(10) - (10)].boolean)); + (yyvsp[-3].into)->rel->relpersistence = (yyvsp[-8].ival); + (yyvsp[-3].into)->skipData = !((yyvsp[0].boolean)); (yyval.node) = (PGNode *) ctas; - ;} + } +#line 30849 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1454: + case 1492: /* CreateAsStmt: CREATE_P OR REPLACE OptTemp TABLE create_as_target AS SelectStmt opt_with_data */ #line 38 "third_party/libpg_query/grammar/statements/create_as.y" - { + { PGCreateTableAsStmt *ctas = makeNode(PGCreateTableAsStmt); - ctas->query = (yyvsp[(8) - (9)].node); - ctas->into = (yyvsp[(6) - (9)].into); + ctas->query = (yyvsp[-1].node); + ctas->into = (yyvsp[-3].into); ctas->relkind = PG_OBJECT_TABLE; ctas->is_select_into = false; ctas->onconflict = PG_REPLACE_ON_CONFLICT; /* cram additional flags into the PGIntoClause */ - (yyvsp[(6) - (9)].into)->rel->relpersistence = (yyvsp[(4) - (9)].ival); - (yyvsp[(6) - (9)].into)->skipData = !((yyvsp[(9) - (9)].boolean)); + (yyvsp[-3].into)->rel->relpersistence = (yyvsp[-5].ival); + (yyvsp[-3].into)->skipData = !((yyvsp[0].boolean)); (yyval.node) = (PGNode *) ctas; - ;} + } +#line 30866 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1455: + case 1493: /* opt_with_data: WITH DATA_P */ #line 54 "third_party/libpg_query/grammar/statements/create_as.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 30872 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1456: + case 1494: /* opt_with_data: WITH NO DATA_P */ #line 55 "third_party/libpg_query/grammar/statements/create_as.y" - { (yyval.boolean) = false; ;} + { (yyval.boolean) = false; } +#line 30878 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1457: + case 1495: /* opt_with_data: %empty */ #line 56 "third_party/libpg_query/grammar/statements/create_as.y" - { (yyval.boolean) = true; ;} + { (yyval.boolean) = true; } +#line 30884 "third_party/libpg_query/grammar/grammar_out.cpp" break; - case 1458: + case 1496: /* create_as_target: qualified_name opt_column_list OptWith OnCommitOption */ #line 62 "third_party/libpg_query/grammar/statements/create_as.y" - { + { (yyval.into) = makeNode(PGIntoClause); - (yyval.into)->rel = (yyvsp[(1) - (4)].range); - (yyval.into)->colNames = (yyvsp[(2) - (4)].list); - (yyval.into)->options = (yyvsp[(3) - (4)].list); - (yyval.into)->onCommit = (yyvsp[(4) - (4)].oncommit); + (yyval.into)->rel = (yyvsp[-3].range); + (yyval.into)->colNames = (yyvsp[-2].list); + (yyval.into)->options = (yyvsp[-1].list); + (yyval.into)->onCommit = (yyvsp[0].oncommit); (yyval.into)->viewQuery = NULL; (yyval.into)->skipData = false; /* might get changed later */ - ;} + } +#line 30898 "third_party/libpg_query/grammar/grammar_out.cpp" break; -/* Line 1267 of yacc.c. */ -#line 29673 "third_party/libpg_query/grammar/grammar_out.cpp" +#line 30902 "third_party/libpg_query/grammar/grammar_out.cpp" + default: break; } - YY_SYMBOL_PRINT ("-> $$ =", yyr1[yyn], &yyval, &yyloc); + /* User semantic actions sometimes alter yychar, and that requires + that yytoken be updated with the new translation. We take the + approach of translating immediately before every use of yytoken. + One alternative is translating here after every semantic action, + but that translation would be missed if the semantic action invokes + YYABORT, YYACCEPT, or YYERROR immediately after altering yychar or + if it invokes YYBACKUP. In the case of YYABORT or YYACCEPT, an + incorrect destructor might then be invoked immediately. In the + case of YYERROR or YYBACKUP, subsequent parser actions might lead + to an incorrect destructor call or verbose syntax error message + before the lookahead is translated. */ + YY_SYMBOL_PRINT ("-> $$ =", YY_CAST (yysymbol_kind_t, yyr1[yyn]), &yyval, &yyloc); YYPOPSTACK (yylen); yylen = 0; - YY_STACK_PRINT (yyss, yyssp); *++yyvsp = yyval; *++yylsp = yyloc; - /* Now `shift' the result of the reduction. Determine what state + /* Now 'shift' the result of the reduction. Determine what state that goes to, based on the state we popped back to and the rule number reduced by. */ - - yyn = yyr1[yyn]; - - yystate = yypgoto[yyn - YYNTOKENS] + *yyssp; - if (0 <= yystate && yystate <= YYLAST && yycheck[yystate] == *yyssp) - yystate = yytable[yystate]; - else - yystate = yydefgoto[yyn - YYNTOKENS]; + { + const int yylhs = yyr1[yyn] - YYNTOKENS; + const int yyi = yypgoto[yylhs] + *yyssp; + yystate = (0 <= yyi && yyi <= YYLAST && yycheck[yyi] == *yyssp + ? yytable[yyi] + : yydefgoto[yylhs]); + } goto yynewstate; -/*------------------------------------. -| yyerrlab -- here on detecting error | -`------------------------------------*/ +/*--------------------------------------. +| yyerrlab -- here on detecting error. | +`--------------------------------------*/ yyerrlab: + /* Make sure we have latest lookahead translation. See comments at + user semantic actions for why this is necessary. */ + yytoken = yychar == YYEMPTY ? YYSYMBOL_YYEMPTY : YYTRANSLATE (yychar); /* If not already recovering from an error, report this error. */ if (!yyerrstatus) { ++yynerrs; -#if ! YYERROR_VERBOSE yyerror (&yylloc, yyscanner, YY_("syntax error")); -#else - { - YYSIZE_T yysize = yysyntax_error (0, yystate, yychar); - if (yymsg_alloc < yysize && yymsg_alloc < YYSTACK_ALLOC_MAXIMUM) - { - YYSIZE_T yyalloc = 2 * yysize; - if (! (yysize <= yyalloc && yyalloc <= YYSTACK_ALLOC_MAXIMUM)) - yyalloc = YYSTACK_ALLOC_MAXIMUM; - if (yymsg != yymsgbuf) - YYSTACK_FREE (yymsg); - yymsg = (char *) YYSTACK_ALLOC (yyalloc); - if (yymsg) - yymsg_alloc = yyalloc; - else - { - yymsg = yymsgbuf; - yymsg_alloc = sizeof yymsgbuf; - } - } - - if (0 < yysize && yysize <= yymsg_alloc) - { - (void) yysyntax_error (yymsg, yystate, yychar); - yyerror (&yylloc, yyscanner, yymsg); - } - else - { - yyerror (&yylloc, yyscanner, YY_("syntax error")); - if (yysize != 0) - goto yyexhaustedlab; - } - } -#endif } - yyerror_range[0] = yylloc; - + yyerror_range[1] = yylloc; if (yyerrstatus == 3) { - /* If just tried and failed to reuse look-ahead token after an - error, discard it. */ + /* If just tried and failed to reuse lookahead token after an + error, discard it. */ if (yychar <= YYEOF) - { - /* Return failure if at end of input. */ - if (yychar == YYEOF) - YYABORT; - } + { + /* Return failure if at end of input. */ + if (yychar == YYEOF) + YYABORT; + } else - { - yydestruct ("Error: discarding", - yytoken, &yylval, &yylloc, yyscanner); - yychar = YYEMPTY; - } + { + yydestruct ("Error: discarding", + yytoken, &yylval, &yylloc, yyscanner); + yychar = YYEMPTY; + } } - /* Else will try to reuse look-ahead token after shifting the error + /* Else will try to reuse lookahead token after shifting the error token. */ goto yyerrlab1; @@ -29771,15 +30978,13 @@ YYLTYPE yylloc; | yyerrorlab -- error raised explicitly by YYERROR. | `---------------------------------------------------*/ yyerrorlab: + /* Pacify compilers when the user code never invokes YYERROR and the + label yyerrorlab therefore never appears in user code. */ + if (0) + YYERROR; + ++yynerrs; - /* Pacify compilers like GCC when the user code never invokes - YYERROR and the label yyerrorlab therefore never appears in user - code. */ - if (/*CONSTCOND*/ 0) - goto yyerrorlab; - - yyerror_range[0] = yylsp[1-yylen]; - /* Do not reclaim the symbols of the rule which action triggered + /* Do not reclaim the symbols of the rule whose action triggered this YYERROR. */ YYPOPSTACK (yylen); yylen = 0; @@ -29792,47 +30997,45 @@ YYLTYPE yylloc; | yyerrlab1 -- common code for both syntax error and YYERROR. | `-------------------------------------------------------------*/ yyerrlab1: - yyerrstatus = 3; /* Each real token shifted decrements this. */ + yyerrstatus = 3; /* Each real token shifted decrements this. */ + /* Pop stack until we find a state that shifts the error token. */ for (;;) { yyn = yypact[yystate]; - if (yyn != YYPACT_NINF) - { - yyn += YYTERROR; - if (0 <= yyn && yyn <= YYLAST && yycheck[yyn] == YYTERROR) - { - yyn = yytable[yyn]; - if (0 < yyn) - break; - } - } + if (!yypact_value_is_default (yyn)) + { + yyn += YYSYMBOL_YYerror; + if (0 <= yyn && yyn <= YYLAST && yycheck[yyn] == YYSYMBOL_YYerror) + { + yyn = yytable[yyn]; + if (0 < yyn) + break; + } + } /* Pop the current state because it cannot handle the error token. */ if (yyssp == yyss) - YYABORT; + YYABORT; - yyerror_range[0] = *yylsp; + yyerror_range[1] = *yylsp; yydestruct ("Error: popping", - yystos[yystate], yyvsp, yylsp, yyscanner); + YY_ACCESSING_SYMBOL (yystate), yyvsp, yylsp, yyscanner); YYPOPSTACK (1); yystate = *yyssp; YY_STACK_PRINT (yyss, yyssp); } - if (yyn == YYFINAL) - YYACCEPT; - + YY_IGNORE_MAYBE_UNINITIALIZED_BEGIN *++yyvsp = yylval; + YY_IGNORE_MAYBE_UNINITIALIZED_END - yyerror_range[1] = yylloc; - /* Using YYLLOC is tempting, but would change the location of - the look-ahead. YYLOC is available though. */ - YYLLOC_DEFAULT (yyloc, (yyerror_range - 1), 2); - *++yylsp = yyloc; + yyerror_range[2] = yylloc; + ++yylsp; + YYLLOC_DEFAULT (*yylsp, yyerror_range, 2); /* Shift the error token. */ - YY_SYMBOL_PRINT ("Shifting", yystos[yyn], yyvsp, yylsp); + YY_SYMBOL_PRINT ("Shifting", YY_ACCESSING_SYMBOL (yyn), yyvsp, yylsp); yystate = yyn; goto yynewstate; @@ -29843,51 +31046,55 @@ YYLTYPE yylloc; `-------------------------------------*/ yyacceptlab: yyresult = 0; - goto yyreturn; + goto yyreturnlab; + /*-----------------------------------. | yyabortlab -- YYABORT comes here. | `-----------------------------------*/ yyabortlab: yyresult = 1; - goto yyreturn; + goto yyreturnlab; -#ifndef yyoverflow -/*-------------------------------------------------. -| yyexhaustedlab -- memory exhaustion comes here. | -`-------------------------------------------------*/ + +/*-----------------------------------------------------------. +| yyexhaustedlab -- YYNOMEM (memory exhaustion) comes here. | +`-----------------------------------------------------------*/ yyexhaustedlab: yyerror (&yylloc, yyscanner, YY_("memory exhausted")); yyresult = 2; - /* Fall through. */ -#endif + goto yyreturnlab; + -yyreturn: - if (yychar != YYEOF && yychar != YYEMPTY) - yydestruct ("Cleanup: discarding lookahead", - yytoken, &yylval, &yylloc, yyscanner); - /* Do not reclaim the symbols of the rule which action triggered +/*----------------------------------------------------------. +| yyreturnlab -- parsing is finished, clean up and return. | +`----------------------------------------------------------*/ +yyreturnlab: + if (yychar != YYEMPTY) + { + /* Make sure we have latest lookahead translation. See comments at + user semantic actions for why this is necessary. */ + yytoken = YYTRANSLATE (yychar); + yydestruct ("Cleanup: discarding lookahead", + yytoken, &yylval, &yylloc, yyscanner); + } + /* Do not reclaim the symbols of the rule whose action triggered this YYABORT or YYACCEPT. */ YYPOPSTACK (yylen); YY_STACK_PRINT (yyss, yyssp); while (yyssp != yyss) { yydestruct ("Cleanup: popping", - yystos[*yyssp], yyvsp, yylsp, yyscanner); + YY_ACCESSING_SYMBOL (+*yyssp), yyvsp, yylsp, yyscanner); YYPOPSTACK (1); } #ifndef yyoverflow if (yyss != yyssa) YYSTACK_FREE (yyss); #endif -#if YYERROR_VERBOSE - if (yymsg != yymsgbuf) - YYSTACK_FREE (yymsg); -#endif - /* Make sure YYID is used. */ - return YYID (yyresult); -} + return yyresult; +} #line 83 "third_party/libpg_query/grammar/statements/create_as.y" @@ -30265,6 +31472,12 @@ insertSelectOptions(PGSelectStmt *stmt, PGWithClause *withClause, core_yyscan_t yyscanner) { + if (stmt->type != T_PGSelectStmt) { + ereport(ERROR, + (errcode(PG_ERRCODE_SYNTAX_ERROR), + errmsg("DESCRIBE/SHOW/SUMMARIZE with CTE/ORDER BY/... not allowed - wrap the statement in a subquery instead"), + parser_errposition(exprLocation((PGNode *) stmt)))); + } Assert(IsA(stmt, PGSelectStmt)); /* @@ -30640,4 +31853,3 @@ parser_init(base_yy_extra_type *yyext) #undef yylloc } // namespace duckdb_libpgquery - diff --git a/src/duckdb/third_party/libpg_query/src_backend_parser_scan.cpp b/src/duckdb/third_party/libpg_query/src_backend_parser_scan.cpp index e9c913804..378637436 100644 --- a/src/duckdb/third_party/libpg_query/src_backend_parser_scan.cpp +++ b/src/duckdb/third_party/libpg_query/src_backend_parser_scan.cpp @@ -624,7 +624,7 @@ struct yy_trans_info flex_int32_t yy_verify; flex_int32_t yy_nxt; }; -static const flex_int16_t yy_accept[297] = +static const flex_int16_t yy_accept[309] = { 0, 0, 0, 12, 12, 0, 0, 0, 0, 11, 11, 0, 0, 0, 0, 0, 0, 0, 0, 55, 55, @@ -637,28 +637,29 @@ static const flex_int16_t yy_accept[297] = 26, 26, 27, 27, 35, 36, 35, 1, 73, 71, 43, 74, 44, 65, 1, 63, 61, 77, 2, 66, - 77, 76, 80, 60, 62, 68, 70, 67, 69, 75, - 82, 8, 20, 18, 59, 15, 12, 9, 9, 10, - 5, 7, 4, 3, 58, 57, 11, 16, 16, 17, - 32, 22, 22, 30, 23, 38, 39, 37, 37, 38, - 31, 46, 45, 47, 53, 53, 55, 24, 24, 25, - 26, 26, 28, 37, 37, 44, 1, 1, 64, 2, - 78, 77, 81, 79, 50, 21, 9, 14, 10, 9, - 3, 16, 13, 17, 16, 22, 41, 23, 22, 39, - 37, 37, 40, 47, 53, 55, 24, 25, 24, 26, - 28, 37, 37, 9, 9, 9, 9, 16, 16, 16, - - 16, 22, 22, 22, 22, 39, 37, 37, 40, 55, - 24, 24, 24, 24, 28, 37, 37, 9, 9, 9, - 9, 9, 16, 16, 16, 16, 16, 22, 22, 22, - 22, 22, 37, 37, 55, 24, 24, 24, 24, 24, - 28, 37, 37, 9, 16, 22, 37, 33, 55, 24, - 28, 37, 34, 37, 55, 28, 37, 37, 55, 55, - 55, 28, 28, 28, 37, 37, 55, 55, 28, 28, - 37, 56, 55, 55, 55, 55, 29, 28, 28, 28, - 28, 55, 55, 55, 55, 55, 28, 28, 28, 28, - 28, 55, 55, 28, 28, 0 - + 77, 76, 80, 0, 60, 62, 68, 70, 67, 69, + 75, 82, 8, 20, 18, 59, 15, 12, 9, 9, + 10, 5, 7, 4, 3, 58, 57, 11, 16, 16, + 17, 32, 22, 22, 30, 23, 38, 39, 37, 37, + 38, 31, 46, 45, 47, 53, 53, 55, 24, 24, + 25, 26, 26, 28, 37, 37, 74, 0, 44, 1, + 1, 64, 77, 0, 2, 78, 77, 81, 79, 76, + 75, 0, 50, 21, 9, 14, 10, 9, 3, 16, + 13, 17, 16, 22, 41, 23, 22, 39, 37, 37, + 40, 47, 53, 55, 24, 25, 24, 26, 28, 37, + + 37, 77, 0, 79, 0, 9, 9, 9, 9, 16, + 16, 16, 16, 22, 22, 22, 22, 39, 37, 37, + 40, 55, 24, 24, 24, 24, 28, 37, 37, 9, + 9, 9, 9, 9, 16, 16, 16, 16, 16, 22, + 22, 22, 22, 22, 37, 37, 55, 24, 24, 24, + 24, 24, 28, 37, 37, 9, 16, 22, 37, 33, + 55, 24, 28, 37, 34, 37, 55, 28, 37, 37, + 55, 55, 55, 28, 28, 28, 37, 37, 55, 55, + 28, 28, 37, 56, 55, 55, 55, 55, 29, 28, + 28, 28, 28, 55, 55, 55, 55, 55, 28, 28, + + 28, 28, 28, 55, 55, 28, 28, 0 } ; static const YY_CHAR yy_ec[256] = @@ -672,10 +673,10 @@ static const YY_CHAR yy_ec[256] = 21, 22, 23, 24, 25, 26, 27, 28, 29, 28, 30, 30, 30, 30, 30, 30, 30, 31, 30, 32, 30, 30, 33, 30, 34, 30, 30, 35, 30, 30, - 11, 36, 11, 8, 30, 24, 25, 26, 27, 28, + 11, 36, 11, 8, 37, 24, 25, 26, 27, 28, 29, 28, 30, 30, 30, 30, 30, 30, 30, 31, - 30, 32, 30, 30, 33, 30, 37, 30, 30, 38, + 30, 32, 30, 30, 33, 30, 38, 30, 30, 39, 30, 30, 1, 24, 1, 24, 1, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, 30, @@ -693,384 +694,392 @@ static const YY_CHAR yy_ec[256] = 30, 30, 30, 30, 30 } ; -static const YY_CHAR yy_meta[39] = +static const YY_CHAR yy_meta[40] = { 0, 1, 1, 2, 2, 3, 4, 5, 3, 3, 6, 1, 7, 3, 3, 1, 7, 8, 8, 1, 3, 3, 3, 1, 3, 9, 9, 9, 9, 9, 10, - 10, 10, 10, 10, 10, 11, 10, 10 + 10, 10, 10, 10, 10, 11, 10, 10, 10 } ; -static const flex_int16_t yy_base[364] = +static const flex_int16_t yy_base[376] = { 0, - 0, 0, 415, 409, 34, 54, 412, 400, 395, 393, - 41, 50, 392, 380, 38, 54, 383, 378, 85, 121, - 373, 367, 157, 193, 37, 46, 376, 1215, 77, 81, - 343, 1215, 83, 0, 0, 1215, 1215, 351, 50, 76, - 53, 80, 83, 84, 339, 339, 90, 0, 338, 337, - 332, 332, 324, 0, 112, 0, 0, 80, 315, 0, - 315, 0, 125, 0, 0, 128, 103, 0, 142, 0, - 0, 312, 312, 1215, 108, 145, 297, 275, 160, 1215, - 163, 173, 283, 260, 1215, 1215, 84, 176, 0, 0, - 1215, 116, 278, 0, 226, 262, 1215, 155, 0, 0, - - 170, 175, 188, 1215, 1215, 0, 0, 0, 0, 133, - 0, 1215, 1215, 1215, 107, 1215, 0, 206, 209, 267, - 0, 141, 1215, 0, 0, 1215, 0, 212, 241, 266, - 0, 250, 255, 1215, 259, 1215, 254, 0, 0, 0, - 0, 0, 1215, 263, 179, 0, 230, 258, 264, 242, - 273, 0, 216, 0, 0, 234, 0, 274, 0, 0, - 1215, 200, 151, 204, 1215, 1215, 289, 1215, 223, 298, - 0, 303, 1215, 222, 306, 312, 1215, 219, 321, 215, - 0, 0, 0, 221, 0, 198, 326, 210, 329, 0, - 176, 0, 0, 335, 342, 348, 351, 355, 364, 368, - - 371, 377, 384, 390, 393, 1215, 0, 0, 1215, 175, - 397, 406, 410, 413, 173, 0, 0, 419, 426, 432, - 435, 441, 448, 454, 457, 461, 470, 474, 477, 483, - 492, 496, 0, 0, 157, 499, 505, 512, 521, 525, - 154, 0, 0, 528, 534, 541, 0, 1215, 125, 547, - 114, 0, 1215, 0, 550, 555, 0, 0, 564, 0, - 122, 569, 0, 89, 0, 0, 61, 578, 42, 583, - 0, 1215, 592, 597, 606, 611, 1215, 620, 625, 634, - 639, 648, 653, 662, 667, 676, 681, 690, 695, 704, - 709, 718, 0, 723, 0, 1215, 737, 748, 759, 770, - - 781, 792, 803, 814, 825, 836, 845, 848, 854, 864, - 875, 886, 897, 908, 918, 929, 940, 947, 953, 963, - 972, 977, 977, 979, 981, 986, 996, 1007, 1011, 1013, - 1022, 1033, 1044, 1048, 1050, 1052, 1061, 1065, 1067, 1076, - 1087, 1098, 1102, 1104, 1113, 1117, 1119, 1121, 1123, 1125, - 1127, 1129, 1131, 1133, 1135, 1137, 1146, 1157, 1161, 1170, - 1181, 1192, 1203 + 0, 0, 486, 484, 35, 55, 483, 475, 466, 465, + 42, 51, 458, 450, 39, 55, 449, 448, 86, 123, + 442, 437, 160, 197, 38, 47, 439, 1250, 78, 82, + 413, 1250, 84, 0, 0, 1250, 1250, 421, 51, 77, + 54, 81, 84, 85, 409, 405, 91, 0, 408, 403, + 402, 401, 395, 0, 109, 0, 0, 81, 385, 0, + 386, 0, 126, 0, 0, 129, 117, 0, 144, 0, + 0, 384, 370, 1250, 112, 163, 361, 338, 166, 1250, + 169, 175, 348, 326, 1250, 1250, 83, 179, 0, 0, + 1250, 167, 342, 0, 231, 325, 1250, 173, 0, 0, + + 178, 188, 195, 118, 1250, 1250, 0, 0, 0, 0, + 197, 0, 1250, 1250, 1250, 132, 1250, 0, 216, 219, + 332, 0, 129, 1250, 0, 0, 1250, 0, 224, 246, + 326, 0, 255, 260, 1250, 325, 1250, 297, 0, 0, + 0, 0, 0, 1250, 297, 184, 0, 269, 263, 269, + 284, 278, 0, 263, 0, 0, 224, 132, 287, 0, + 279, 0, 268, 135, 0, 1250, 289, 142, 272, 293, + 295, 258, 1250, 1250, 313, 1250, 264, 317, 0, 331, + 1250, 254, 334, 340, 1250, 240, 349, 229, 0, 0, + 0, 230, 0, 205, 354, 210, 357, 0, 189, 0, + + 0, 348, 307, 352, 311, 370, 376, 379, 384, 392, + 397, 400, 405, 413, 418, 421, 426, 1250, 0, 0, + 1250, 172, 434, 439, 442, 447, 167, 0, 0, 455, + 460, 463, 468, 476, 481, 484, 489, 498, 502, 505, + 511, 518, 527, 531, 0, 0, 157, 534, 540, 547, + 556, 560, 143, 0, 0, 563, 569, 576, 0, 1250, + 115, 582, 93, 0, 1250, 0, 585, 590, 0, 0, + 599, 0, 105, 604, 0, 90, 0, 0, 62, 613, + 43, 618, 0, 1250, 627, 632, 641, 646, 1250, 655, + 660, 669, 674, 683, 688, 697, 702, 711, 716, 725, + + 730, 739, 744, 753, 0, 758, 0, 1250, 772, 783, + 794, 805, 816, 827, 838, 849, 860, 871, 880, 883, + 889, 899, 910, 921, 932, 943, 953, 964, 975, 982, + 988, 998, 1007, 1012, 1012, 1014, 1016, 1021, 1031, 1042, + 1046, 1048, 1057, 1068, 1079, 1083, 1085, 1087, 1096, 1100, + 1102, 1111, 1122, 1133, 1137, 1139, 1148, 1152, 1154, 1156, + 1158, 1160, 1162, 1164, 1166, 1168, 1170, 1172, 1181, 1192, + 1196, 1205, 1216, 1227, 1238 } ; -static const flex_int16_t yy_def[364] = +static const flex_int16_t yy_def[376] = { 0, - 296, 1, 297, 297, 298, 298, 299, 299, 300, 300, - 301, 301, 302, 302, 303, 303, 299, 299, 304, 304, - 302, 302, 305, 305, 306, 306, 296, 296, 296, 296, - 307, 296, 308, 307, 307, 296, 296, 307, 307, 296, - 307, 296, 296, 307, 307, 307, 296, 309, 309, 309, - 309, 309, 309, 310, 296, 311, 311, 296, 296, 312, - 296, 313, 296, 314, 314, 296, 315, 316, 296, 317, - 317, 318, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 307, 307, - 296, 296, 319, 307, 320, 307, 296, 296, 321, 307, - - 296, 296, 296, 296, 296, 307, 307, 307, 307, 296, - 309, 296, 296, 296, 296, 296, 310, 296, 296, 296, - 311, 296, 296, 322, 312, 296, 313, 296, 296, 296, - 314, 296, 296, 296, 296, 296, 296, 323, 324, 325, - 316, 317, 296, 326, 296, 327, 296, 296, 296, 296, - 296, 328, 296, 329, 330, 319, 320, 320, 307, 321, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 331, - 322, 296, 296, 296, 332, 296, 296, 296, 333, 296, - 334, 335, 336, 326, 327, 296, 296, 296, 337, 328, - 296, 338, 339, 340, 331, 331, 331, 341, 332, 332, - - 332, 342, 333, 333, 333, 296, 343, 344, 296, 296, - 345, 337, 337, 337, 296, 346, 347, 340, 340, 296, - 340, 331, 341, 341, 296, 341, 332, 342, 342, 296, - 342, 333, 348, 349, 296, 345, 345, 296, 345, 337, - 296, 350, 351, 340, 341, 342, 352, 296, 296, 345, - 296, 353, 296, 354, 296, 296, 355, 356, 296, 357, - 296, 296, 358, 296, 359, 349, 296, 360, 296, 361, - 351, 296, 360, 360, 362, 360, 296, 361, 361, 363, - 361, 360, 360, 296, 360, 360, 361, 361, 296, 361, - 361, 362, 357, 363, 358, 0, 296, 296, 296, 296, - - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296 + 308, 1, 309, 309, 310, 310, 311, 311, 312, 312, + 313, 313, 314, 314, 315, 315, 311, 311, 316, 316, + 314, 314, 317, 317, 318, 318, 308, 308, 308, 308, + 319, 308, 320, 319, 319, 308, 308, 319, 319, 308, + 319, 308, 308, 319, 319, 319, 308, 321, 321, 321, + 321, 321, 321, 322, 308, 323, 323, 308, 308, 324, + 308, 325, 308, 326, 326, 308, 327, 328, 308, 329, + 329, 330, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 319, 319, + 308, 308, 331, 319, 332, 319, 308, 308, 333, 319, + + 308, 308, 308, 308, 308, 308, 319, 319, 319, 319, + 308, 321, 308, 308, 308, 308, 308, 322, 308, 308, + 308, 323, 308, 308, 334, 324, 308, 325, 308, 308, + 308, 326, 308, 308, 308, 308, 308, 308, 335, 336, + 337, 328, 329, 308, 338, 308, 339, 308, 308, 308, + 308, 308, 340, 308, 341, 342, 308, 308, 331, 332, + 332, 319, 308, 308, 333, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 343, 334, 308, + 308, 308, 344, 308, 308, 308, 345, 308, 346, 347, + 348, 338, 339, 308, 308, 308, 349, 340, 308, 350, + + 351, 308, 308, 308, 308, 352, 343, 343, 343, 353, + 344, 344, 344, 354, 345, 345, 345, 308, 355, 356, + 308, 308, 357, 349, 349, 349, 308, 358, 359, 352, + 352, 308, 352, 343, 353, 353, 308, 353, 344, 354, + 354, 308, 354, 345, 360, 361, 308, 357, 357, 308, + 357, 349, 308, 362, 363, 352, 353, 354, 364, 308, + 308, 357, 308, 365, 308, 366, 308, 308, 367, 368, + 308, 369, 308, 308, 370, 308, 371, 361, 308, 372, + 308, 373, 363, 308, 372, 372, 374, 372, 308, 373, + 373, 375, 373, 372, 372, 308, 372, 372, 373, 373, + + 308, 373, 373, 374, 369, 375, 370, 0, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308 } ; -static const flex_int16_t yy_nxt[1254] = +static const flex_int16_t yy_nxt[1290] = { 0, 28, 29, 30, 29, 31, 32, 33, 34, 35, 36, 37, 38, 34, 39, 40, 41, 42, 42, 43, 44, 45, 46, 47, 35, 48, 49, 48, 48, 50, 48, - 51, 48, 48, 52, 53, 28, 52, 53, 57, 86, - 71, 57, 57, 65, 72, 58, 57, 57, 86, 59, - 66, 277, 65, 57, 57, 57, 71, 57, 57, 66, - 72, 57, 57, 95, 99, 58, 57, 57, 100, 59, - 272, 96, 87, 57, 57, 57, 67, 57, 88, 88, - 88, 87, 88, 88, 88, 67, 75, 76, 75, 91, - 97, 122, 98, 98, 101, 123, 102, 102, 77, 92, - - 92, 104, 270, 105, 106, 107, 110, 110, 103, 145, - 145, 145, 165, 118, 119, 119, 166, 154, 78, 137, - 155, 78, 75, 76, 75, 120, 128, 129, 129, 132, - 133, 133, 92, 92, 77, 268, 138, 134, 130, 139, - 140, 135, 256, 132, 133, 133, 145, 145, 145, 110, - 110, 134, 122, 255, 78, 135, 123, 78, 81, 82, - 81, 148, 149, 149, 151, 151, 151, 164, 164, 134, - 83, 98, 98, 150, 151, 151, 151, 88, 88, 88, - 145, 145, 145, 103, 161, 251, 162, 162, 249, 101, - 84, 102, 102, 84, 81, 82, 81, 241, 103, 235, - - 163, 163, 215, 103, 164, 164, 83, 118, 119, 119, - 167, 167, 167, 128, 129, 129, 162, 162, 168, 120, - 164, 164, 169, 211, 210, 130, 84, 143, 103, 84, - 158, 206, 202, 158, 158, 198, 194, 158, 158, 158, - 91, 158, 172, 172, 172, 158, 158, 158, 191, 158, - 173, 132, 133, 133, 174, 189, 176, 176, 176, 148, - 149, 149, 186, 135, 177, 187, 187, 187, 178, 143, - 180, 150, 179, 177, 151, 151, 151, 188, 158, 175, - 170, 158, 158, 159, 91, 158, 158, 158, 153, 158, - 167, 167, 167, 158, 158, 158, 152, 158, 168, 196, - - 119, 119, 169, 147, 172, 172, 172, 200, 129, 129, - 146, 197, 173, 176, 176, 176, 174, 126, 143, 201, - 126, 177, 204, 133, 133, 178, 124, 187, 187, 187, - 213, 149, 149, 116, 205, 177, 219, 220, 220, 188, - 115, 114, 214, 196, 119, 119, 113, 112, 221, 196, - 119, 119, 196, 119, 119, 197, 224, 225, 225, 109, - 108, 197, 94, 90, 222, 200, 129, 129, 226, 200, - 129, 129, 200, 129, 129, 296, 79, 201, 229, 230, - 230, 201, 79, 73, 227, 204, 133, 133, 73, 69, - 231, 204, 133, 133, 204, 133, 133, 205, 237, 238, - - 238, 69, 63, 205, 63, 61, 232, 213, 149, 149, - 239, 213, 149, 149, 213, 149, 149, 61, 55, 214, - 219, 220, 220, 214, 55, 296, 240, 219, 220, 220, - 296, 296, 221, 167, 167, 167, 219, 220, 220, 221, - 296, 168, 196, 119, 119, 169, 296, 296, 244, 224, - 225, 225, 296, 296, 222, 224, 225, 225, 172, 172, - 172, 226, 224, 225, 225, 296, 173, 226, 296, 296, - 174, 200, 129, 129, 245, 229, 230, 230, 229, 230, - 230, 296, 296, 227, 176, 176, 176, 231, 296, 296, - 231, 296, 177, 229, 230, 230, 178, 204, 133, 133, - - 237, 238, 238, 296, 296, 246, 237, 238, 238, 232, - 296, 296, 239, 187, 187, 187, 296, 296, 239, 296, - 296, 177, 237, 238, 238, 188, 213, 149, 149, 219, - 220, 220, 296, 296, 250, 224, 225, 225, 240, 296, - 296, 244, 229, 230, 230, 296, 296, 245, 237, 238, - 238, 259, 259, 259, 246, 296, 262, 262, 262, 260, - 250, 296, 296, 261, 263, 259, 259, 259, 264, 296, - 262, 262, 262, 260, 296, 296, 296, 261, 263, 274, - 259, 259, 264, 296, 279, 262, 262, 275, 296, 296, - 296, 276, 280, 274, 259, 259, 281, 296, 274, 259, - - 259, 275, 296, 296, 296, 276, 275, 283, 284, 284, - 276, 296, 274, 259, 259, 275, 296, 296, 296, 285, - 275, 279, 262, 262, 286, 296, 279, 262, 262, 280, - 296, 296, 296, 281, 280, 288, 289, 289, 281, 296, - 279, 262, 262, 280, 296, 296, 296, 290, 280, 274, - 259, 259, 291, 296, 274, 259, 259, 292, 296, 296, - 296, 276, 292, 259, 259, 259, 276, 296, 274, 259, - 259, 293, 296, 296, 296, 261, 292, 274, 259, 259, - 286, 296, 279, 262, 262, 275, 296, 296, 296, 286, - 294, 279, 262, 262, 281, 296, 262, 262, 262, 294, - - 296, 296, 296, 281, 295, 279, 262, 262, 264, 296, - 279, 262, 262, 294, 296, 296, 296, 291, 280, 283, - 284, 284, 291, 296, 288, 289, 289, 275, 296, 296, - 296, 285, 280, 296, 296, 296, 290, 54, 54, 54, - 54, 54, 54, 54, 54, 54, 54, 54, 56, 56, - 56, 56, 56, 56, 56, 56, 56, 56, 56, 60, - 60, 60, 60, 60, 60, 60, 60, 60, 60, 60, - 62, 62, 62, 62, 62, 62, 62, 62, 62, 62, - 62, 64, 64, 64, 64, 64, 64, 64, 64, 64, - 64, 64, 68, 68, 68, 68, 68, 68, 68, 68, - - 68, 68, 68, 70, 70, 70, 70, 70, 70, 70, - 70, 70, 70, 70, 74, 74, 74, 74, 74, 74, - 74, 74, 74, 74, 74, 80, 80, 80, 80, 80, - 80, 80, 80, 80, 80, 80, 85, 85, 85, 85, - 85, 85, 85, 85, 85, 85, 85, 89, 296, 296, - 296, 89, 93, 296, 296, 93, 93, 93, 111, 296, - 296, 111, 111, 111, 117, 117, 117, 117, 117, 296, - 117, 117, 117, 117, 117, 121, 121, 121, 121, 121, - 121, 296, 121, 121, 121, 121, 125, 125, 125, 296, - 125, 125, 125, 125, 125, 125, 125, 127, 127, 127, - - 127, 127, 296, 127, 127, 127, 127, 127, 131, 131, - 131, 131, 131, 296, 131, 131, 131, 131, 136, 136, - 136, 136, 136, 136, 136, 136, 136, 136, 136, 141, - 141, 141, 141, 141, 296, 141, 141, 141, 141, 141, - 142, 142, 142, 142, 296, 142, 142, 142, 142, 142, - 142, 144, 296, 296, 296, 144, 144, 156, 296, 296, - 156, 156, 156, 157, 296, 157, 157, 157, 157, 157, - 157, 157, 157, 157, 160, 296, 296, 296, 160, 171, - 296, 296, 296, 171, 181, 181, 182, 182, 183, 183, - 184, 296, 296, 184, 184, 184, 185, 296, 185, 185, - - 185, 185, 185, 185, 185, 185, 185, 190, 296, 190, - 190, 190, 190, 190, 190, 190, 190, 190, 192, 192, - 193, 193, 195, 195, 195, 195, 195, 195, 195, 195, - 195, 195, 195, 199, 199, 199, 199, 199, 199, 199, - 199, 199, 199, 199, 203, 203, 203, 203, 203, 203, - 203, 203, 203, 203, 203, 207, 207, 208, 208, 209, - 209, 212, 212, 212, 212, 212, 212, 212, 212, 212, - 212, 212, 216, 216, 217, 217, 218, 218, 218, 218, - 218, 218, 218, 218, 218, 218, 218, 223, 223, 223, - 223, 223, 223, 223, 223, 223, 223, 223, 228, 228, - - 228, 228, 228, 228, 228, 228, 228, 228, 228, 233, - 233, 234, 234, 236, 236, 236, 236, 236, 236, 236, - 236, 236, 236, 236, 242, 242, 243, 243, 247, 247, - 248, 248, 252, 252, 253, 253, 254, 254, 257, 257, - 258, 258, 265, 265, 266, 266, 267, 267, 267, 267, - 267, 296, 267, 267, 267, 267, 267, 269, 269, 269, - 269, 269, 296, 269, 269, 269, 269, 269, 271, 271, - 273, 273, 273, 273, 273, 273, 273, 273, 273, 273, - 273, 278, 278, 278, 278, 278, 278, 278, 278, 278, - 278, 278, 282, 282, 282, 282, 282, 282, 282, 282, - - 282, 282, 282, 287, 287, 287, 287, 287, 287, 287, - 287, 287, 287, 287, 27, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296 + 51, 48, 48, 52, 53, 28, 48, 52, 53, 57, + 86, 71, 57, 57, 65, 72, 58, 57, 57, 86, + 59, 66, 289, 65, 57, 57, 57, 71, 57, 57, + 66, 72, 57, 57, 95, 99, 58, 57, 57, 100, + 59, 284, 96, 87, 57, 57, 57, 67, 57, 88, + 88, 88, 87, 88, 88, 88, 67, 75, 76, 75, + 91, 97, 123, 98, 98, 101, 124, 102, 102, 77, + + 92, 92, 105, 282, 106, 107, 108, 111, 111, 103, + 119, 120, 120, 146, 146, 146, 155, 104, 280, 78, + 156, 268, 121, 78, 75, 76, 75, 129, 130, 130, + 133, 134, 134, 138, 170, 170, 77, 173, 135, 131, + 123, 174, 136, 267, 124, 133, 134, 134, 157, 157, + 139, 163, 163, 135, 140, 141, 78, 136, 169, 169, + 78, 81, 82, 81, 146, 146, 146, 149, 150, 150, + 152, 152, 152, 83, 263, 135, 152, 152, 152, 151, + 88, 88, 88, 157, 157, 146, 146, 146, 261, 163, + 163, 253, 166, 84, 167, 167, 247, 84, 81, 82, + + 81, 103, 101, 158, 102, 102, 103, 168, 168, 164, + 83, 169, 169, 171, 171, 227, 103, 119, 120, 120, + 175, 175, 175, 223, 104, 129, 130, 130, 176, 121, + 84, 222, 177, 172, 84, 161, 144, 131, 161, 161, + 157, 157, 161, 161, 161, 218, 161, 180, 180, 180, + 161, 161, 161, 214, 161, 181, 133, 134, 134, 182, + 158, 184, 184, 184, 149, 150, 150, 210, 136, 185, + 195, 195, 195, 186, 171, 171, 151, 206, 185, 152, + 152, 152, 196, 161, 163, 163, 161, 161, 204, 204, + 161, 161, 161, 91, 161, 199, 103, 197, 161, 161, + + 161, 194, 161, 144, 164, 202, 202, 101, 205, 102, + 102, 171, 171, 188, 175, 175, 175, 103, 208, 120, + 120, 103, 176, 202, 202, 203, 177, 204, 204, 104, + 209, 172, 180, 180, 180, 212, 130, 130, 187, 183, + 181, 184, 184, 184, 182, 178, 162, 213, 91, 185, + 216, 134, 134, 186, 154, 195, 195, 195, 225, 150, + 150, 153, 217, 185, 202, 202, 148, 196, 204, 204, + 226, 231, 232, 232, 147, 127, 103, 208, 120, 120, + 208, 120, 120, 233, 203, 208, 120, 120, 205, 209, + 144, 127, 209, 236, 237, 237, 125, 234, 212, 130, + + 130, 212, 130, 130, 117, 238, 212, 130, 130, 116, + 213, 115, 114, 213, 241, 242, 242, 113, 239, 216, + 134, 134, 216, 134, 134, 110, 243, 216, 134, 134, + 109, 217, 94, 90, 217, 249, 250, 250, 308, 244, + 225, 150, 150, 225, 150, 150, 79, 251, 225, 150, + 150, 79, 226, 73, 73, 226, 231, 232, 232, 69, + 252, 231, 232, 232, 175, 175, 175, 69, 233, 231, + 232, 232, 176, 233, 63, 63, 177, 208, 120, 120, + 61, 256, 236, 237, 237, 236, 237, 237, 61, 234, + 180, 180, 180, 55, 238, 55, 308, 238, 181, 236, + + 237, 237, 182, 212, 130, 130, 241, 242, 242, 308, + 308, 257, 241, 242, 242, 239, 308, 308, 243, 184, + 184, 184, 308, 308, 243, 308, 308, 185, 241, 242, + 242, 186, 216, 134, 134, 249, 250, 250, 308, 308, + 258, 249, 250, 250, 244, 308, 308, 251, 195, 195, + 195, 308, 308, 251, 308, 308, 185, 249, 250, 250, + 196, 225, 150, 150, 231, 232, 232, 308, 308, 262, + 236, 237, 237, 252, 308, 308, 256, 241, 242, 242, + 308, 308, 257, 249, 250, 250, 271, 271, 271, 258, + 308, 274, 274, 274, 272, 262, 308, 308, 273, 275, + + 271, 271, 271, 276, 308, 274, 274, 274, 272, 308, + 308, 308, 273, 275, 286, 271, 271, 276, 308, 291, + 274, 274, 287, 308, 308, 308, 288, 292, 286, 271, + 271, 293, 308, 286, 271, 271, 287, 308, 308, 308, + 288, 287, 295, 296, 296, 288, 308, 286, 271, 271, + 287, 308, 308, 308, 297, 287, 291, 274, 274, 298, + 308, 291, 274, 274, 292, 308, 308, 308, 293, 292, + 300, 301, 301, 293, 308, 291, 274, 274, 292, 308, + 308, 308, 302, 292, 286, 271, 271, 303, 308, 286, + 271, 271, 304, 308, 308, 308, 288, 304, 271, 271, + + 271, 288, 308, 286, 271, 271, 305, 308, 308, 308, + 273, 304, 286, 271, 271, 298, 308, 291, 274, 274, + 287, 308, 308, 308, 298, 306, 291, 274, 274, 293, + 308, 274, 274, 274, 306, 308, 308, 308, 293, 307, + 291, 274, 274, 276, 308, 291, 274, 274, 306, 308, + 308, 308, 303, 292, 295, 296, 296, 303, 308, 300, + 301, 301, 287, 308, 308, 308, 297, 292, 308, 308, + 308, 302, 54, 54, 54, 54, 54, 54, 54, 54, + 54, 54, 54, 56, 56, 56, 56, 56, 56, 56, + 56, 56, 56, 56, 60, 60, 60, 60, 60, 60, + + 60, 60, 60, 60, 60, 62, 62, 62, 62, 62, + 62, 62, 62, 62, 62, 62, 64, 64, 64, 64, + 64, 64, 64, 64, 64, 64, 64, 68, 68, 68, + 68, 68, 68, 68, 68, 68, 68, 68, 70, 70, + 70, 70, 70, 70, 70, 70, 70, 70, 70, 74, + 74, 74, 74, 74, 74, 74, 74, 74, 74, 74, + 80, 80, 80, 80, 80, 80, 80, 80, 80, 80, + 80, 85, 85, 85, 85, 85, 85, 85, 85, 85, + 85, 85, 89, 308, 308, 308, 89, 93, 308, 308, + 93, 93, 93, 112, 308, 308, 112, 112, 112, 118, + + 118, 118, 118, 118, 308, 118, 118, 118, 118, 118, + 122, 122, 122, 122, 122, 122, 308, 122, 122, 122, + 122, 126, 126, 126, 308, 126, 126, 126, 126, 126, + 126, 126, 128, 128, 128, 128, 128, 308, 128, 128, + 128, 128, 128, 132, 132, 132, 132, 132, 308, 132, + 132, 132, 132, 137, 137, 137, 137, 137, 137, 137, + 137, 137, 137, 137, 142, 142, 142, 142, 142, 308, + 142, 142, 142, 142, 142, 143, 143, 143, 143, 308, + 143, 143, 143, 143, 143, 143, 145, 308, 308, 308, + 145, 145, 159, 308, 308, 159, 159, 159, 160, 308, + + 160, 160, 160, 160, 160, 160, 160, 160, 160, 165, + 308, 308, 308, 165, 179, 308, 308, 308, 179, 189, + 189, 190, 190, 191, 191, 192, 308, 308, 192, 192, + 192, 193, 308, 193, 193, 193, 193, 193, 193, 193, + 193, 193, 198, 308, 198, 198, 198, 198, 198, 198, + 198, 198, 198, 200, 200, 201, 201, 207, 207, 207, + 207, 207, 207, 207, 207, 207, 207, 207, 211, 211, + 211, 211, 211, 211, 211, 211, 211, 211, 211, 215, + 215, 215, 215, 215, 215, 215, 215, 215, 215, 215, + 219, 219, 220, 220, 221, 221, 224, 224, 224, 224, + + 224, 224, 224, 224, 224, 224, 224, 228, 228, 229, + 229, 230, 230, 230, 230, 230, 230, 230, 230, 230, + 230, 230, 235, 235, 235, 235, 235, 235, 235, 235, + 235, 235, 235, 240, 240, 240, 240, 240, 240, 240, + 240, 240, 240, 240, 245, 245, 246, 246, 248, 248, + 248, 248, 248, 248, 248, 248, 248, 248, 248, 254, + 254, 255, 255, 259, 259, 260, 260, 264, 264, 265, + 265, 266, 266, 269, 269, 270, 270, 277, 277, 278, + 278, 279, 279, 279, 279, 279, 308, 279, 279, 279, + 279, 279, 281, 281, 281, 281, 281, 308, 281, 281, + + 281, 281, 281, 283, 283, 285, 285, 285, 285, 285, + 285, 285, 285, 285, 285, 285, 290, 290, 290, 290, + 290, 290, 290, 290, 290, 290, 290, 294, 294, 294, + 294, 294, 294, 294, 294, 294, 294, 294, 299, 299, + 299, 299, 299, 299, 299, 299, 299, 299, 299, 27, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308 } ; -static const flex_int16_t yy_chk[1254] = +static const flex_int16_t yy_chk[1290] = { 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, 1, 1, 5, 25, - 15, 5, 5, 11, 15, 5, 5, 5, 26, 5, - 11, 269, 12, 5, 5, 5, 16, 5, 6, 12, - 16, 6, 6, 39, 41, 6, 6, 6, 41, 6, - 267, 39, 25, 6, 6, 6, 11, 6, 29, 29, - 29, 26, 30, 30, 30, 12, 19, 19, 19, 33, - 40, 58, 40, 40, 42, 58, 42, 42, 19, 33, - - 33, 43, 264, 43, 44, 44, 47, 47, 42, 75, - 75, 75, 115, 55, 55, 55, 115, 87, 19, 67, - 87, 19, 20, 20, 20, 55, 63, 63, 63, 66, - 66, 66, 92, 92, 20, 261, 67, 66, 63, 67, - 67, 66, 251, 69, 69, 69, 76, 76, 76, 110, - 110, 69, 122, 249, 20, 69, 122, 20, 23, 23, - 23, 79, 79, 79, 81, 81, 81, 163, 163, 79, - 23, 98, 98, 79, 82, 82, 82, 88, 88, 88, - 145, 145, 145, 98, 101, 241, 101, 101, 235, 102, - 23, 102, 102, 23, 24, 24, 24, 215, 101, 210, - - 103, 103, 191, 102, 103, 103, 24, 118, 118, 118, - 119, 119, 119, 128, 128, 128, 162, 162, 119, 118, - 164, 164, 119, 188, 186, 128, 24, 184, 162, 24, - 95, 180, 178, 95, 95, 174, 169, 95, 95, 95, - 156, 95, 129, 129, 129, 95, 95, 95, 153, 95, - 129, 132, 132, 132, 129, 150, 133, 133, 133, 148, - 148, 148, 147, 132, 133, 149, 149, 149, 133, 144, - 137, 148, 135, 149, 151, 151, 151, 149, 158, 130, - 120, 158, 158, 96, 93, 158, 158, 158, 84, 158, - 167, 167, 167, 158, 158, 158, 83, 158, 167, 170, - - 170, 170, 167, 78, 172, 172, 172, 175, 175, 175, - 77, 170, 172, 176, 176, 176, 172, 73, 72, 175, - 61, 176, 179, 179, 179, 176, 59, 187, 187, 187, - 189, 189, 189, 53, 179, 187, 194, 194, 194, 187, - 52, 51, 189, 195, 195, 195, 50, 49, 194, 196, - 196, 196, 197, 197, 197, 195, 198, 198, 198, 46, - 45, 196, 38, 31, 197, 199, 199, 199, 198, 200, - 200, 200, 201, 201, 201, 27, 22, 199, 202, 202, - 202, 200, 21, 18, 201, 203, 203, 203, 17, 14, - 202, 204, 204, 204, 205, 205, 205, 203, 211, 211, - - 211, 13, 10, 204, 9, 8, 205, 212, 212, 212, - 211, 213, 213, 213, 214, 214, 214, 7, 4, 212, - 218, 218, 218, 213, 3, 0, 214, 219, 219, 219, - 0, 0, 218, 220, 220, 220, 221, 221, 221, 219, - 0, 220, 222, 222, 222, 220, 0, 0, 221, 223, - 223, 223, 0, 0, 222, 224, 224, 224, 225, 225, - 225, 223, 226, 226, 226, 0, 225, 224, 0, 0, - 225, 227, 227, 227, 226, 228, 228, 228, 229, 229, - 229, 0, 0, 227, 230, 230, 230, 228, 0, 0, - 229, 0, 230, 231, 231, 231, 230, 232, 232, 232, - - 236, 236, 236, 0, 0, 231, 237, 237, 237, 232, - 0, 0, 236, 238, 238, 238, 0, 0, 237, 0, - 0, 238, 239, 239, 239, 238, 240, 240, 240, 244, - 244, 244, 0, 0, 239, 245, 245, 245, 240, 0, - 0, 244, 246, 246, 246, 0, 0, 245, 250, 250, - 250, 255, 255, 255, 246, 0, 256, 256, 256, 255, - 250, 0, 0, 255, 256, 259, 259, 259, 256, 0, - 262, 262, 262, 259, 0, 0, 0, 259, 262, 268, - 268, 268, 262, 0, 270, 270, 270, 268, 0, 0, - 0, 268, 270, 273, 273, 273, 270, 0, 274, 274, - - 274, 273, 0, 0, 0, 273, 274, 275, 275, 275, - 274, 0, 276, 276, 276, 275, 0, 0, 0, 275, - 276, 278, 278, 278, 276, 0, 279, 279, 279, 278, - 0, 0, 0, 278, 279, 280, 280, 280, 279, 0, - 281, 281, 281, 280, 0, 0, 0, 280, 281, 282, - 282, 282, 281, 0, 283, 283, 283, 282, 0, 0, - 0, 282, 283, 284, 284, 284, 283, 0, 285, 285, - 285, 284, 0, 0, 0, 284, 285, 286, 286, 286, - 285, 0, 287, 287, 287, 286, 0, 0, 0, 286, - 287, 288, 288, 288, 287, 0, 289, 289, 289, 288, - - 0, 0, 0, 288, 289, 290, 290, 290, 289, 0, - 291, 291, 291, 290, 0, 0, 0, 290, 291, 292, - 292, 292, 291, 0, 294, 294, 294, 292, 0, 0, - 0, 292, 294, 0, 0, 0, 294, 297, 297, 297, - 297, 297, 297, 297, 297, 297, 297, 297, 298, 298, - 298, 298, 298, 298, 298, 298, 298, 298, 298, 299, - 299, 299, 299, 299, 299, 299, 299, 299, 299, 299, - 300, 300, 300, 300, 300, 300, 300, 300, 300, 300, - 300, 301, 301, 301, 301, 301, 301, 301, 301, 301, - 301, 301, 302, 302, 302, 302, 302, 302, 302, 302, - - 302, 302, 302, 303, 303, 303, 303, 303, 303, 303, - 303, 303, 303, 303, 304, 304, 304, 304, 304, 304, - 304, 304, 304, 304, 304, 305, 305, 305, 305, 305, - 305, 305, 305, 305, 305, 305, 306, 306, 306, 306, - 306, 306, 306, 306, 306, 306, 306, 307, 0, 0, - 0, 307, 308, 0, 0, 308, 308, 308, 309, 0, - 0, 309, 309, 309, 310, 310, 310, 310, 310, 0, - 310, 310, 310, 310, 310, 311, 311, 311, 311, 311, - 311, 0, 311, 311, 311, 311, 312, 312, 312, 0, - 312, 312, 312, 312, 312, 312, 312, 313, 313, 313, - - 313, 313, 0, 313, 313, 313, 313, 313, 314, 314, - 314, 314, 314, 0, 314, 314, 314, 314, 315, 315, + 1, 1, 1, 1, 1, 1, 1, 1, 1, 5, + 25, 15, 5, 5, 11, 15, 5, 5, 5, 26, + 5, 11, 281, 12, 5, 5, 5, 16, 5, 6, + 12, 16, 6, 6, 39, 41, 6, 6, 6, 41, + 6, 279, 39, 25, 6, 6, 6, 11, 6, 29, + 29, 29, 26, 30, 30, 30, 12, 19, 19, 19, + 33, 40, 58, 40, 40, 42, 58, 42, 42, 19, + + 33, 33, 43, 276, 43, 44, 44, 47, 47, 42, + 55, 55, 55, 75, 75, 75, 87, 42, 273, 19, + 87, 263, 55, 19, 20, 20, 20, 63, 63, 63, + 66, 66, 66, 67, 104, 104, 20, 116, 66, 63, + 123, 116, 66, 261, 123, 69, 69, 69, 158, 158, + 67, 164, 164, 69, 67, 67, 20, 69, 168, 168, + 20, 23, 23, 23, 76, 76, 76, 79, 79, 79, + 81, 81, 81, 23, 253, 79, 82, 82, 82, 79, + 88, 88, 88, 92, 92, 146, 146, 146, 247, 98, + 98, 227, 101, 23, 101, 101, 222, 23, 24, 24, + + 24, 98, 102, 92, 102, 102, 101, 103, 103, 98, + 24, 103, 103, 111, 111, 199, 102, 119, 119, 119, + 120, 120, 120, 196, 102, 129, 129, 129, 120, 119, + 24, 194, 120, 111, 24, 95, 192, 129, 95, 95, + 157, 157, 95, 95, 95, 188, 95, 130, 130, 130, + 95, 95, 95, 186, 95, 130, 133, 133, 133, 130, + 157, 134, 134, 134, 149, 149, 149, 182, 133, 134, + 150, 150, 150, 134, 172, 172, 149, 177, 150, 152, + 152, 152, 150, 161, 163, 163, 161, 161, 169, 169, + 161, 161, 161, 159, 161, 154, 163, 151, 161, 161, + + 161, 148, 161, 145, 163, 167, 167, 170, 169, 170, + 170, 171, 171, 138, 175, 175, 175, 167, 178, 178, + 178, 170, 175, 203, 203, 167, 175, 205, 205, 170, + 178, 171, 180, 180, 180, 183, 183, 183, 136, 131, + 180, 184, 184, 184, 180, 121, 96, 183, 93, 184, + 187, 187, 187, 184, 84, 195, 195, 195, 197, 197, + 197, 83, 187, 195, 202, 202, 78, 195, 204, 204, + 197, 206, 206, 206, 77, 73, 202, 207, 207, 207, + 208, 208, 208, 206, 202, 209, 209, 209, 204, 207, + 72, 61, 208, 210, 210, 210, 59, 209, 211, 211, + + 211, 212, 212, 212, 53, 210, 213, 213, 213, 52, + 211, 51, 50, 212, 214, 214, 214, 49, 213, 215, + 215, 215, 216, 216, 216, 46, 214, 217, 217, 217, + 45, 215, 38, 31, 216, 223, 223, 223, 27, 217, + 224, 224, 224, 225, 225, 225, 22, 223, 226, 226, + 226, 21, 224, 18, 17, 225, 230, 230, 230, 14, + 226, 231, 231, 231, 232, 232, 232, 13, 230, 233, + 233, 233, 232, 231, 10, 9, 232, 234, 234, 234, + 8, 233, 235, 235, 235, 236, 236, 236, 7, 234, + 237, 237, 237, 4, 235, 3, 0, 236, 237, 238, + + 238, 238, 237, 239, 239, 239, 240, 240, 240, 0, + 0, 238, 241, 241, 241, 239, 0, 0, 240, 242, + 242, 242, 0, 0, 241, 0, 0, 242, 243, 243, + 243, 242, 244, 244, 244, 248, 248, 248, 0, 0, + 243, 249, 249, 249, 244, 0, 0, 248, 250, 250, + 250, 0, 0, 249, 0, 0, 250, 251, 251, 251, + 250, 252, 252, 252, 256, 256, 256, 0, 0, 251, + 257, 257, 257, 252, 0, 0, 256, 258, 258, 258, + 0, 0, 257, 262, 262, 262, 267, 267, 267, 258, + 0, 268, 268, 268, 267, 262, 0, 0, 267, 268, + + 271, 271, 271, 268, 0, 274, 274, 274, 271, 0, + 0, 0, 271, 274, 280, 280, 280, 274, 0, 282, + 282, 282, 280, 0, 0, 0, 280, 282, 285, 285, + 285, 282, 0, 286, 286, 286, 285, 0, 0, 0, + 285, 286, 287, 287, 287, 286, 0, 288, 288, 288, + 287, 0, 0, 0, 287, 288, 290, 290, 290, 288, + 0, 291, 291, 291, 290, 0, 0, 0, 290, 291, + 292, 292, 292, 291, 0, 293, 293, 293, 292, 0, + 0, 0, 292, 293, 294, 294, 294, 293, 0, 295, + 295, 295, 294, 0, 0, 0, 294, 295, 296, 296, + + 296, 295, 0, 297, 297, 297, 296, 0, 0, 0, + 296, 297, 298, 298, 298, 297, 0, 299, 299, 299, + 298, 0, 0, 0, 298, 299, 300, 300, 300, 299, + 0, 301, 301, 301, 300, 0, 0, 0, 300, 301, + 302, 302, 302, 301, 0, 303, 303, 303, 302, 0, + 0, 0, 302, 303, 304, 304, 304, 303, 0, 306, + 306, 306, 304, 0, 0, 0, 304, 306, 0, 0, + 0, 306, 309, 309, 309, 309, 309, 309, 309, 309, + 309, 309, 309, 310, 310, 310, 310, 310, 310, 310, + 310, 310, 310, 310, 311, 311, 311, 311, 311, 311, + + 311, 311, 311, 311, 311, 312, 312, 312, 312, 312, + 312, 312, 312, 312, 312, 312, 313, 313, 313, 313, + 313, 313, 313, 313, 313, 313, 313, 314, 314, 314, + 314, 314, 314, 314, 314, 314, 314, 314, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 315, 316, - 316, 316, 316, 316, 0, 316, 316, 316, 316, 316, - 317, 317, 317, 317, 0, 317, 317, 317, 317, 317, - 317, 318, 0, 0, 0, 318, 318, 319, 0, 0, - 319, 319, 319, 320, 0, 320, 320, 320, 320, 320, - 320, 320, 320, 320, 321, 0, 0, 0, 321, 322, - 0, 0, 0, 322, 323, 323, 324, 324, 325, 325, - 326, 0, 0, 326, 326, 326, 327, 0, 327, 327, - - 327, 327, 327, 327, 327, 327, 327, 328, 0, 328, - 328, 328, 328, 328, 328, 328, 328, 328, 329, 329, - 330, 330, 331, 331, 331, 331, 331, 331, 331, 331, - 331, 331, 331, 332, 332, 332, 332, 332, 332, 332, - 332, 332, 332, 332, 333, 333, 333, 333, 333, 333, - 333, 333, 333, 333, 333, 334, 334, 335, 335, 336, - 336, 337, 337, 337, 337, 337, 337, 337, 337, 337, - 337, 337, 338, 338, 339, 339, 340, 340, 340, 340, - 340, 340, 340, 340, 340, 340, 340, 341, 341, 341, - 341, 341, 341, 341, 341, 341, 341, 341, 342, 342, - - 342, 342, 342, 342, 342, 342, 342, 342, 342, 343, - 343, 344, 344, 345, 345, 345, 345, 345, 345, 345, - 345, 345, 345, 345, 346, 346, 347, 347, 348, 348, - 349, 349, 350, 350, 351, 351, 352, 352, 353, 353, - 354, 354, 355, 355, 356, 356, 357, 357, 357, 357, - 357, 0, 357, 357, 357, 357, 357, 358, 358, 358, - 358, 358, 0, 358, 358, 358, 358, 358, 359, 359, - 360, 360, 360, 360, 360, 360, 360, 360, 360, 360, - 360, 361, 361, 361, 361, 361, 361, 361, 361, 361, - 361, 361, 362, 362, 362, 362, 362, 362, 362, 362, - - 362, 362, 362, 363, 363, 363, 363, 363, 363, 363, - 363, 363, 363, 363, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296, 296, 296, 296, 296, 296, 296, 296, - 296, 296, 296 + 316, 316, 316, 316, 316, 316, 316, 316, 316, 316, + 317, 317, 317, 317, 317, 317, 317, 317, 317, 317, + 317, 318, 318, 318, 318, 318, 318, 318, 318, 318, + 318, 318, 319, 0, 0, 0, 319, 320, 0, 0, + 320, 320, 320, 321, 0, 0, 321, 321, 321, 322, + + 322, 322, 322, 322, 0, 322, 322, 322, 322, 322, + 323, 323, 323, 323, 323, 323, 0, 323, 323, 323, + 323, 324, 324, 324, 0, 324, 324, 324, 324, 324, + 324, 324, 325, 325, 325, 325, 325, 0, 325, 325, + 325, 325, 325, 326, 326, 326, 326, 326, 0, 326, + 326, 326, 326, 327, 327, 327, 327, 327, 327, 327, + 327, 327, 327, 327, 328, 328, 328, 328, 328, 0, + 328, 328, 328, 328, 328, 329, 329, 329, 329, 0, + 329, 329, 329, 329, 329, 329, 330, 0, 0, 0, + 330, 330, 331, 0, 0, 331, 331, 331, 332, 0, + + 332, 332, 332, 332, 332, 332, 332, 332, 332, 333, + 0, 0, 0, 333, 334, 0, 0, 0, 334, 335, + 335, 336, 336, 337, 337, 338, 0, 0, 338, 338, + 338, 339, 0, 339, 339, 339, 339, 339, 339, 339, + 339, 339, 340, 0, 340, 340, 340, 340, 340, 340, + 340, 340, 340, 341, 341, 342, 342, 343, 343, 343, + 343, 343, 343, 343, 343, 343, 343, 343, 344, 344, + 344, 344, 344, 344, 344, 344, 344, 344, 344, 345, + 345, 345, 345, 345, 345, 345, 345, 345, 345, 345, + 346, 346, 347, 347, 348, 348, 349, 349, 349, 349, + + 349, 349, 349, 349, 349, 349, 349, 350, 350, 351, + 351, 352, 352, 352, 352, 352, 352, 352, 352, 352, + 352, 352, 353, 353, 353, 353, 353, 353, 353, 353, + 353, 353, 353, 354, 354, 354, 354, 354, 354, 354, + 354, 354, 354, 354, 355, 355, 356, 356, 357, 357, + 357, 357, 357, 357, 357, 357, 357, 357, 357, 358, + 358, 359, 359, 360, 360, 361, 361, 362, 362, 363, + 363, 364, 364, 365, 365, 366, 366, 367, 367, 368, + 368, 369, 369, 369, 369, 369, 0, 369, 369, 369, + 369, 369, 370, 370, 370, 370, 370, 0, 370, 370, + + 370, 370, 370, 371, 371, 372, 372, 372, 372, 372, + 372, 372, 372, 372, 372, 372, 373, 373, 373, 373, + 373, 373, 373, 373, 373, 373, 373, 374, 374, 374, + 374, 374, 374, 374, 374, 374, 374, 374, 375, 375, + 375, 375, 375, 375, 375, 375, 375, 375, 375, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308, 308, + 308, 308, 308, 308, 308, 308, 308, 308, 308 } ; /* The intent behind this definition is that it'll catch @@ -1151,7 +1160,7 @@ static void check_escape_warning(core_yyscan_t yyscanner); extern int core_yyget_column(yyscan_t yyscanner); extern void core_yyset_column(int column_no, yyscan_t yyscanner); -#line 1153 "third_party/libpg_query/src_backend_parser_scan.cpp" +#line 1162 "third_party/libpg_query/src_backend_parser_scan.cpp" #define YY_NO_INPUT 1 /* * OK, here is a short description of lex/flex rules behavior. @@ -1305,7 +1314,7 @@ extern void core_yyset_column(int column_no, yyscan_t yyscanner); * Note that xcstart must appear before operator, as explained above! * Also whitespace (comment) must appear before operator. */ -#line 1307 "third_party/libpg_query/src_backend_parser_scan.cpp" +#line 1316 "third_party/libpg_query/src_backend_parser_scan.cpp" #define INITIAL 0 #define xb 1 @@ -1587,7 +1596,7 @@ YY_DECL #line 404 "third_party/libpg_query/scan.l" -#line 1596 "third_party/libpg_query/src_backend_parser_scan.cpp" +#line 1605 "third_party/libpg_query/src_backend_parser_scan.cpp" while ( /*CONSTCOND*/1 ) /* loops until end-of-file is reached */ { @@ -1614,13 +1623,13 @@ YY_DECL while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state ) { yy_current_state = (int) yy_def[yy_current_state]; - if ( yy_current_state >= 297 ) + if ( yy_current_state >= 309 ) yy_c = yy_meta[yy_c]; } yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c]; ++yy_cp; } - while ( yy_current_state != 296 ); + while ( yy_current_state != 308 ); yy_cp = yyg->yy_last_accepting_cpos; yy_current_state = yyg->yy_last_accepting_state; @@ -2658,7 +2667,7 @@ YY_RULE_SETUP #line 1089 "third_party/libpg_query/scan.l" YY_FATAL_ERROR( "flex scanner jammed" ); YY_BREAK -#line 2667 "third_party/libpg_query/src_backend_parser_scan.cpp" +#line 2676 "third_party/libpg_query/src_backend_parser_scan.cpp" case YY_END_OF_BUFFER: { @@ -2956,7 +2965,7 @@ static int yy_get_next_buffer (yyscan_t yyscanner) while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state ) { yy_current_state = (int) yy_def[yy_current_state]; - if ( yy_current_state >= 297 ) + if ( yy_current_state >= 309 ) yy_c = yy_meta[yy_c]; } yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c]; @@ -2985,11 +2994,11 @@ static int yy_get_next_buffer (yyscan_t yyscanner) while ( yy_chk[yy_base[yy_current_state] + yy_c] != yy_current_state ) { yy_current_state = (int) yy_def[yy_current_state]; - if ( yy_current_state >= 297 ) + if ( yy_current_state >= 309 ) yy_c = yy_meta[yy_c]; } yy_current_state = yy_nxt[yy_base[yy_current_state] + yy_c]; - yy_is_jam = (yy_current_state == 296); + yy_is_jam = (yy_current_state == 308); (void)yyg; return yy_is_jam ? 0 : yy_current_state; @@ -3983,6 +3992,28 @@ litbufdup(core_yyscan_t yyscanner) static int process_integer_literal(const char *token, YYSTYPE *lval) { + int underscores = 0; + int len = 0; + for(char* cursor = (char*)token; *cursor != '\0'; cursor++) + { + len++; + if(*cursor == '_') + underscores++; + } + + if(underscores != 0) + { + char* new_token = (char*)palloc(len - underscores + 1); + char* cursor = new_token; + for(char* old_cursor = (char*)token; *old_cursor != '\0'; old_cursor++) + { + if(*old_cursor != '_') + *cursor++ = *old_cursor; + } + *cursor = '\0'; + token = new_token; + } + long val; char *endptr; diff --git a/src/duckdb/third_party/mbedtls/library/entropy_poll.cpp b/src/duckdb/third_party/mbedtls/library/entropy_poll.cpp index 058c307df..c894fb6b3 100644 --- a/src/duckdb/third_party/mbedtls/library/entropy_poll.cpp +++ b/src/duckdb/third_party/mbedtls/library/entropy_poll.cpp @@ -43,7 +43,7 @@ #if !defined(unix) && !defined(__unix__) && !defined(__unix) && \ !defined(__APPLE__) && !defined(_WIN32) && !defined(__QNXNTO__) && \ - !defined(__HAIKU__) && !defined(__midipix__) + !defined(__HAIKU__) && !defined(__midipix__) && !defined(__MVS__) #error "Platform entropy sources only work on Unix and Windows, see MBEDTLS_NO_PLATFORM_ENTROPY in mbedtls_config.h" #endif diff --git a/src/duckdb/third_party/pcg/pcg_uint128.hpp b/src/duckdb/third_party/pcg/pcg_uint128.hpp index c085d0ee2..a29f77cbf 100644 --- a/src/duckdb/third_party/pcg/pcg_uint128.hpp +++ b/src/duckdb/third_party/pcg/pcg_uint128.hpp @@ -67,7 +67,7 @@ #define PCG_LITTLE_ENDIAN 1 #elif __BIG_ENDIAN__ || _BIG_ENDIAN #define PCG_LITTLE_ENDIAN 0 - #elif __x86_64 || __x86_64__ || _M_X64 || __i386 || __i386__ || _M_IX86 + #elif __x86_64 || __x86_64__ || _M_X64 || __i386 || __i386__ || _M_IX86 || _M_ARM || _M_ARM64 #define PCG_LITTLE_ENDIAN 1 #elif __powerpc__ || __POWERPC__ || __ppc__ || __PPC__ \ || __m68k__ || __mc68000__ diff --git a/src/duckdb/third_party/thrift/thrift/thrift-config.h b/src/duckdb/third_party/thrift/thrift/thrift-config.h index 19e473cf6..e4761f8d1 100644 --- a/src/duckdb/third_party/thrift/thrift/thrift-config.h +++ b/src/duckdb/third_party/thrift/thrift/thrift-config.h @@ -22,7 +22,7 @@ #ifdef _WIN32 -#if defined(_M_IX86) || defined(_M_X64) +#if defined(_M_IX86) || defined(_M_X64) || defined(_M_ARM) || defined(_M_ARM64) #define ARITHMETIC_RIGHT_SHIFT 1 #define SIGNED_RIGHT_SHIFT_IS 1 #endif diff --git a/src/duckdb/ub_src_common.cpp b/src/duckdb/ub_src_common.cpp index 0eac32719..8145f7005 100644 --- a/src/duckdb/ub_src_common.cpp +++ b/src/duckdb/ub_src_common.cpp @@ -40,7 +40,7 @@ #include "src/common/multi_file_reader.cpp" -#include "src/common/preserved_error.cpp" +#include "src/common/error_data.cpp" #include "src/common/printer.cpp" diff --git a/src/duckdb/ub_src_common_exception.cpp b/src/duckdb/ub_src_common_exception.cpp new file mode 100644 index 000000000..a5659f043 --- /dev/null +++ b/src/duckdb/ub_src_common_exception.cpp @@ -0,0 +1,6 @@ +#include "src/common/exception/binder_exception.cpp" + +#include "src/common/exception/catalog_exception.cpp" + +#include "src/common/exception/parser_exception.cpp" + diff --git a/src/duckdb/ub_src_common_types.cpp b/src/duckdb/ub_src_common_types.cpp index 4e2e2ca6c..e9ac7bbe6 100644 --- a/src/duckdb/ub_src_common_types.cpp +++ b/src/duckdb/ub_src_common_types.cpp @@ -6,8 +6,6 @@ #include "src/common/types/cast_helpers.cpp" -#include "src/common/types/chunk_collection.cpp" - #include "src/common/types/conflict_manager.cpp" #include "src/common/types/conflict_info.cpp" @@ -22,6 +20,8 @@ #include "src/common/types/hugeint.cpp" +#include "src/common/types/uhugeint.cpp" + #include "src/common/types/uuid.cpp" #include "src/common/types/hyperloglog.cpp" diff --git a/src/duckdb/ub_src_core_functions_scalar_blob.cpp b/src/duckdb/ub_src_core_functions_scalar_blob.cpp index d046d878c..d1ae2334d 100644 --- a/src/duckdb/ub_src_core_functions_scalar_blob.cpp +++ b/src/duckdb/ub_src_core_functions_scalar_blob.cpp @@ -1,4 +1,6 @@ #include "src/core_functions/scalar/blob/base64.cpp" +#include "src/core_functions/scalar/blob/create_sort_key.cpp" + #include "src/core_functions/scalar/blob/encode.cpp" diff --git a/src/duckdb/ub_src_core_functions_scalar_list.cpp b/src/duckdb/ub_src_core_functions_scalar_list.cpp index faf1ea33d..d1002800d 100644 --- a/src/duckdb/ub_src_core_functions_scalar_list.cpp +++ b/src/duckdb/ub_src_core_functions_scalar_list.cpp @@ -14,6 +14,8 @@ #include "src/core_functions/scalar/list/list_inner_product.cpp" +#include "src/core_functions/scalar/list/list_reduce.cpp" + #include "src/core_functions/scalar/list/list_transform.cpp" #include "src/core_functions/scalar/list/list_value.cpp" diff --git a/src/duckdb/ub_src_core_functions_scalar_secret.cpp b/src/duckdb/ub_src_core_functions_scalar_secret.cpp new file mode 100644 index 000000000..772e4301f --- /dev/null +++ b/src/duckdb/ub_src_core_functions_scalar_secret.cpp @@ -0,0 +1,2 @@ +#include "src/core_functions/scalar/secret/which_secret.cpp" + diff --git a/src/duckdb/ub_src_core_functions_scalar_string.cpp b/src/duckdb/ub_src_core_functions_scalar_string.cpp index a833c0d2c..dfff1fee2 100644 --- a/src/duckdb/ub_src_core_functions_scalar_string.cpp +++ b/src/duckdb/ub_src_core_functions_scalar_string.cpp @@ -26,8 +26,12 @@ #include "src/core_functions/scalar/string/pad.cpp" +#include "src/core_functions/scalar/string/parse_path.cpp" + #include "src/core_functions/scalar/string/printf.cpp" +#include "src/core_functions/scalar/string/regexp_escape.cpp" + #include "src/core_functions/scalar/string/repeat.cpp" #include "src/core_functions/scalar/string/replace.cpp" diff --git a/src/duckdb/ub_src_execution_index.cpp b/src/duckdb/ub_src_execution_index.cpp index 32dd0bf7b..6d5926bd6 100644 --- a/src/duckdb/ub_src_execution_index.cpp +++ b/src/duckdb/ub_src_execution_index.cpp @@ -2,3 +2,7 @@ #include "src/execution/index/fixed_size_buffer.cpp" +#include "src/execution/index/unknown_index.cpp" + +#include "src/execution/index/index_type_set.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner.cpp deleted file mode 100644 index 117072ef2..000000000 --- a/src/duckdb/ub_src_execution_operator_csv_scanner.cpp +++ /dev/null @@ -1,18 +0,0 @@ -#include "src/execution/operator/csv_scanner/base_csv_reader.cpp" - -#include "src/execution/operator/csv_scanner/buffered_csv_reader.cpp" - -#include "src/execution/operator/csv_scanner/csv_buffer.cpp" - -#include "src/execution/operator/csv_scanner/csv_buffer_manager.cpp" - -#include "src/execution/operator/csv_scanner/csv_file_handle.cpp" - -#include "src/execution/operator/csv_scanner/csv_reader_options.cpp" - -#include "src/execution/operator/csv_scanner/csv_state_machine.cpp" - -#include "src/execution/operator/csv_scanner/csv_state_machine_cache.cpp" - -#include "src/execution/operator/csv_scanner/parallel_csv_reader.cpp" - diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_buffer_manager.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_buffer_manager.cpp new file mode 100644 index 000000000..707472b8e --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_buffer_manager.cpp @@ -0,0 +1,6 @@ +#include "src/execution/operator/csv_scanner/buffer_manager/csv_buffer.cpp" + +#include "src/execution/operator/csv_scanner/buffer_manager/csv_buffer_manager.cpp" + +#include "src/execution/operator/csv_scanner/buffer_manager/csv_file_handle.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_scanner.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_scanner.cpp new file mode 100644 index 000000000..2f534e908 --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_scanner.cpp @@ -0,0 +1,10 @@ +#include "src/execution/operator/csv_scanner/scanner/base_scanner.cpp" + +#include "src/execution/operator/csv_scanner/scanner/column_count_scanner.cpp" + +#include "src/execution/operator/csv_scanner/scanner/scanner_boundary.cpp" + +#include "src/execution/operator/csv_scanner/scanner/skip_scanner.cpp" + +#include "src/execution/operator/csv_scanner/scanner/string_value_scanner.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_state_machine.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_state_machine.cpp new file mode 100644 index 000000000..fd9d3fece --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_state_machine.cpp @@ -0,0 +1,4 @@ +#include "src/execution/operator/csv_scanner/state_machine/csv_state_machine.cpp" + +#include "src/execution/operator/csv_scanner/state_machine/csv_state_machine_cache.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_table_function.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_table_function.cpp new file mode 100644 index 000000000..b5215a21d --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_table_function.cpp @@ -0,0 +1,4 @@ +#include "src/execution/operator/csv_scanner/table_function/csv_file_scanner.cpp" + +#include "src/execution/operator/csv_scanner/table_function/global_csv_state.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_csv_scanner_util.cpp b/src/duckdb/ub_src_execution_operator_csv_scanner_util.cpp new file mode 100644 index 000000000..7deec5a78 --- /dev/null +++ b/src/duckdb/ub_src_execution_operator_csv_scanner_util.cpp @@ -0,0 +1,4 @@ +#include "src/execution/operator/csv_scanner/util/csv_error.cpp" + +#include "src/execution/operator/csv_scanner/util/csv_reader_options.cpp" + diff --git a/src/duckdb/ub_src_execution_operator_helper.cpp b/src/duckdb/ub_src_execution_operator_helper.cpp index eaa5f1f1a..0821c06b2 100644 --- a/src/duckdb/ub_src_execution_operator_helper.cpp +++ b/src/duckdb/ub_src_execution_operator_helper.cpp @@ -1,5 +1,7 @@ #include "src/execution/operator/helper/physical_batch_collector.cpp" +#include "src/execution/operator/helper/physical_create_secret.cpp" + #include "src/execution/operator/helper/physical_execute.cpp" #include "src/execution/operator/helper/physical_explain_analyze.cpp" diff --git a/src/duckdb/ub_src_execution_operator_join.cpp b/src/duckdb/ub_src_execution_operator_join.cpp index a82b848b9..d7003bd86 100644 --- a/src/duckdb/ub_src_execution_operator_join.cpp +++ b/src/duckdb/ub_src_execution_operator_join.cpp @@ -10,6 +10,8 @@ #include "src/execution/operator/join/physical_delim_join.cpp" +#include "src/execution/operator/join/physical_left_delim_join.cpp" + #include "src/execution/operator/join/physical_hash_join.cpp" #include "src/execution/operator/join/physical_iejoin.cpp" @@ -26,3 +28,5 @@ #include "src/execution/operator/join/physical_range_join.cpp" +#include "src/execution/operator/join/physical_right_delim_join.cpp" + diff --git a/src/duckdb/ub_src_execution_physical_plan.cpp b/src/duckdb/ub_src_execution_physical_plan.cpp index b19bc3722..f077a92cd 100644 --- a/src/duckdb/ub_src_execution_physical_plan.cpp +++ b/src/duckdb/ub_src_execution_physical_plan.cpp @@ -18,6 +18,8 @@ #include "src/execution/physical_plan/plan_create_table.cpp" +#include "src/execution/physical_plan/plan_create_secret.cpp" + #include "src/execution/physical_plan/plan_cross_product.cpp" #include "src/execution/physical_plan/plan_delete.cpp" @@ -68,8 +70,6 @@ #include "src/execution/physical_plan/plan_sample.cpp" -#include "src/execution/physical_plan/plan_show_select.cpp" - #include "src/execution/physical_plan/plan_simple.cpp" #include "src/execution/physical_plan/plan_top_n.cpp" diff --git a/src/duckdb/ub_src_function_table.cpp b/src/duckdb/ub_src_function_table.cpp index 20b479bc3..0250ec7e1 100644 --- a/src/duckdb/ub_src_function_table.cpp +++ b/src/duckdb/ub_src_function_table.cpp @@ -18,6 +18,8 @@ #include "src/function/table/sniff_csv.cpp" +#include "src/function/table/read_file.cpp" + #include "src/function/table/system_functions.cpp" #include "src/function/table/summary.cpp" diff --git a/src/duckdb/ub_src_function_table_system.cpp b/src/duckdb/ub_src_function_table_system.cpp index 62ca411d0..05edcd725 100644 --- a/src/duckdb/ub_src_function_table_system.cpp +++ b/src/duckdb/ub_src_function_table_system.cpp @@ -18,6 +18,8 @@ #include "src/function/table/system/duckdb_schemas.cpp" +#include "src/function/table/system/duckdb_secrets.cpp" + #include "src/function/table/system/duckdb_sequences.cpp" #include "src/function/table/system/duckdb_settings.cpp" diff --git a/src/duckdb/ub_src_main_secret.cpp b/src/duckdb/ub_src_main_secret.cpp new file mode 100644 index 000000000..b9c15c6ff --- /dev/null +++ b/src/duckdb/ub_src_main_secret.cpp @@ -0,0 +1,6 @@ +#include "src/main/secret/secret.cpp" + +#include "src/main/secret/secret_manager.cpp" + +#include "src/main/secret/secret_storage.cpp" + diff --git a/src/duckdb/ub_src_optimizer_pushdown.cpp b/src/duckdb/ub_src_optimizer_pushdown.cpp index ce710b452..211fccb97 100644 --- a/src/duckdb/ub_src_optimizer_pushdown.cpp +++ b/src/duckdb/ub_src_optimizer_pushdown.cpp @@ -18,6 +18,8 @@ #include "src/optimizer/pushdown/pushdown_projection.cpp" +#include "src/optimizer/pushdown/pushdown_semi_anti_join.cpp" + #include "src/optimizer/pushdown/pushdown_set_operation.cpp" #include "src/optimizer/pushdown/pushdown_single_join.cpp" diff --git a/src/duckdb/ub_src_parser_parsed_data.cpp b/src/duckdb/ub_src_parser_parsed_data.cpp index 49a6198ae..eec89a7b0 100644 --- a/src/duckdb/ub_src_parser_parsed_data.cpp +++ b/src/duckdb/ub_src_parser_parsed_data.cpp @@ -8,6 +8,8 @@ #include "src/parser/parsed_data/attach_info.cpp" +#include "src/parser/parsed_data/comment_on_info.cpp" + #include "src/parser/parsed_data/create_info.cpp" #include "src/parser/parsed_data/create_index_info.cpp" @@ -22,6 +24,8 @@ #include "src/parser/parsed_data/create_pragma_function_info.cpp" +#include "src/parser/parsed_data/create_secret_info.cpp" + #include "src/parser/parsed_data/create_sequence_info.cpp" #include "src/parser/parsed_data/create_scalar_function_info.cpp" @@ -38,6 +42,8 @@ #include "src/parser/parsed_data/drop_info.cpp" +#include "src/parser/parsed_data/extra_drop_info.cpp" + #include "src/parser/parsed_data/sample_options.cpp" #include "src/parser/parsed_data/transaction_info.cpp" diff --git a/src/duckdb/ub_src_parser_statement.cpp b/src/duckdb/ub_src_parser_statement.cpp index ee2ba5f45..a2f3d2b5c 100644 --- a/src/duckdb/ub_src_parser_statement.cpp +++ b/src/duckdb/ub_src_parser_statement.cpp @@ -40,8 +40,6 @@ #include "src/parser/statement/set_statement.cpp" -#include "src/parser/statement/show_statement.cpp" - #include "src/parser/statement/transaction_statement.cpp" #include "src/parser/statement/update_statement.cpp" diff --git a/src/duckdb/ub_src_parser_tableref.cpp b/src/duckdb/ub_src_parser_tableref.cpp index edebf17db..b6075a887 100644 --- a/src/duckdb/ub_src_parser_tableref.cpp +++ b/src/duckdb/ub_src_parser_tableref.cpp @@ -8,6 +8,8 @@ #include "src/parser/tableref/pivotref.cpp" +#include "src/parser/tableref/showref.cpp" + #include "src/parser/tableref/subqueryref.cpp" #include "src/parser/tableref/table_function.cpp" diff --git a/src/duckdb/ub_src_parser_transform_statement.cpp b/src/duckdb/ub_src_parser_transform_statement.cpp index 1f97b399f..83df9d167 100644 --- a/src/duckdb/ub_src_parser_transform_statement.cpp +++ b/src/duckdb/ub_src_parser_transform_statement.cpp @@ -10,6 +10,8 @@ #include "src/parser/transform/statement/transform_create_function.cpp" +#include "src/parser/transform/statement/transform_comment_on.cpp" + #include "src/parser/transform/statement/transform_copy.cpp" #include "src/parser/transform/statement/transform_copy_database.cpp" @@ -44,6 +46,8 @@ #include "src/parser/transform/statement/transform_rename.cpp" +#include "src/parser/transform/statement/transform_secret.cpp" + #include "src/parser/transform/statement/transform_select.cpp" #include "src/parser/transform/statement/transform_select_node.cpp" diff --git a/src/duckdb/ub_src_planner_binder_statement.cpp b/src/duckdb/ub_src_planner_binder_statement.cpp index f642c9747..fc5303edb 100644 --- a/src/duckdb/ub_src_planner_binder_statement.cpp +++ b/src/duckdb/ub_src_planner_binder_statement.cpp @@ -48,5 +48,3 @@ #include "src/planner/binder/statement/bind_vacuum.cpp" -#include "src/planner/binder/statement/bind_show.cpp" - diff --git a/src/duckdb/ub_src_planner_binder_tableref.cpp b/src/duckdb/ub_src_planner_binder_tableref.cpp index b2a9a54d5..f07091a23 100644 --- a/src/duckdb/ub_src_planner_binder_tableref.cpp +++ b/src/duckdb/ub_src_planner_binder_tableref.cpp @@ -8,6 +8,8 @@ #include "src/planner/binder/tableref/bind_pivot.cpp" +#include "src/planner/binder/tableref/bind_showref.cpp" + #include "src/planner/binder/tableref/bind_subqueryref.cpp" #include "src/planner/binder/tableref/bind_table_function.cpp" diff --git a/src/duckdb/ub_src_planner_filter.cpp b/src/duckdb/ub_src_planner_filter.cpp index c6a118bb2..057a07714 100644 --- a/src/duckdb/ub_src_planner_filter.cpp +++ b/src/duckdb/ub_src_planner_filter.cpp @@ -4,3 +4,5 @@ #include "src/planner/filter/null_filter.cpp" +#include "src/planner/filter/struct_filter.cpp" + diff --git a/src/duckdb/ub_src_storage.cpp b/src/duckdb/ub_src_storage.cpp index f8dca4ef1..040bb11b1 100644 --- a/src/duckdb/ub_src_storage.cpp +++ b/src/duckdb/ub_src_storage.cpp @@ -4,6 +4,8 @@ #include "src/storage/checkpoint_manager.cpp" +#include "src/storage/temporary_memory_manager.cpp" + #include "src/storage/block.cpp" #include "src/storage/data_pointer.cpp" diff --git a/src/duckdb/ub_src_storage_compression.cpp b/src/duckdb/ub_src_storage_compression.cpp index f6f348202..ded26082e 100644 --- a/src/duckdb/ub_src_storage_compression.cpp +++ b/src/duckdb/ub_src_storage_compression.cpp @@ -18,5 +18,7 @@ #include "src/storage/compression/patas.cpp" +#include "src/storage/compression/alprd.cpp" + #include "src/storage/compression/fsst.cpp" diff --git a/src/duckdb/ub_src_storage_compression_alp.cpp b/src/duckdb/ub_src_storage_compression_alp.cpp new file mode 100644 index 000000000..7c4a9ceab --- /dev/null +++ b/src/duckdb/ub_src_storage_compression_alp.cpp @@ -0,0 +1,4 @@ +#include "src/storage/compression/alp/alp_constants.cpp" + +#include "src/storage/compression/alp/alp.cpp" + diff --git a/src/duckdb/ub_src_storage_serialization.cpp b/src/duckdb/ub_src_storage_serialization.cpp index 2465830ff..3e6bfd736 100644 --- a/src/duckdb/ub_src_storage_serialization.cpp +++ b/src/duckdb/ub_src_storage_serialization.cpp @@ -4,6 +4,8 @@ #include "src/storage/serialization/serialize_expression.cpp" +#include "src/storage/serialization/serialize_extra_drop_info.cpp" + #include "src/storage/serialization/serialize_logical_operator.cpp" #include "src/storage/serialization/serialize_macro_function.cpp" diff --git a/src/include/sources.mk b/src/include/sources.mk index 6803ea24e..d134e9c62 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_catalog_entry_dependency.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/skiplist/SkipList.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/aes.o duckdb/third_party/mbedtls/library/aria.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/camellia.o duckdb/third_party/mbedtls/library/cipher.o duckdb/third_party/mbedtls/library/cipher_wrap.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/entropy.o duckdb/third_party/mbedtls/library/entropy_poll.o duckdb/third_party/mbedtls/library/gcm.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/column_reader.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/parquet_crypto.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/serialize_parquet.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_catalog_entry_dependency.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_exception.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_secret.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_buffer_manager.o duckdb/ub_src_execution_operator_csv_scanner_scanner.o duckdb/ub_src_execution_operator_csv_scanner_sniffer.o duckdb/ub_src_execution_operator_csv_scanner_state_machine.o duckdb/ub_src_execution_operator_csv_scanner_table_function.o duckdb/ub_src_execution_operator_csv_scanner_util.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_secret.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_alp.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/fetch_row_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/skiplist/SkipList.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/aes.o duckdb/third_party/mbedtls/library/aria.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/camellia.o duckdb/third_party/mbedtls/library/cipher.o duckdb/third_party/mbedtls/library/cipher_wrap.o duckdb/third_party/mbedtls/library/constant_time.o duckdb/third_party/mbedtls/library/entropy.o duckdb/third_party/mbedtls/library/entropy_poll.o duckdb/third_party/mbedtls/library/gcm.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/column_reader.o duckdb/extension/parquet/column_writer.o duckdb/extension/parquet/parquet_crypto.o duckdb/extension/parquet/parquet_extension.o duckdb/extension/parquet/parquet_metadata.o duckdb/extension/parquet/parquet_reader.o duckdb/extension/parquet/parquet_statistics.o duckdb/extension/parquet/parquet_timestamp.o duckdb/extension/parquet/parquet_writer.o duckdb/extension/parquet/serialize_parquet.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